From: PatrikBuhr Date: Tue, 6 Sep 2022 14:57:49 +0000 (+0200) Subject: Added support for S3 object store X-Git-Tag: 1.2.0~16 X-Git-Url: https://gerrit.o-ran-sc.org/r/gitweb?a=commitdiff_plain;h=refs%2Fchanges%2F05%2F9005%2F5;p=nonrtric%2Fplt%2Fdmaapadapter.git Added support for S3 object store PM rop files can be received from file or from s3 object store. If the file ready event contains a field "objectStoreBucket", the file will be fetched from there instead. Change-Id: Ia2b27a9a7fb41ebf4fde87df09d656ffab616209 Signed-off-by: PatrikBuhr Issue-ID: NONRTRIC-773 --- diff --git a/config/application.yaml b/config/application.yaml index 0bd063c..86e5987 100644 --- a/config/application.yaml +++ b/config/application.yaml @@ -76,6 +76,12 @@ app: # several redundant boostrap servers can be specified, separated by a comma ','. kafka: bootstrap-servers: localhost:9092 + # The maximum number of records returned in a single call to poll() (default 100) + max-poll-records: 500 # If the file name is empty, no authorization token is used auth-token-file: pm-files-path: /tmp + s3: + endpointOverride: http://localhost:9000 + accessKeyId: minio + secretAccessKey: miniostorage diff --git a/pom.xml b/pom.xml index ffe624f..fb3135b 100644 --- a/pom.xml +++ b/pom.xml @@ -59,8 +59,25 @@ 3.7.0.1746 0.8.5 true + 4.0.0-rc-2 + 1.4 + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + com.googlecode.protobuf-java-format + protobuf-java-format + ${protobuf-java-format.version} + + + com.google.protobuf + protobuf-java-util + 3.21.5 + org.springframework.boot spring-boot-starter-web @@ -187,6 +204,16 @@ guava 31.0.1-jre + + software.amazon.awssdk + s3 + 2.13.73 + + + com.amazonaws + aws-java-sdk + 1.11.795 + diff --git a/src/main/java/org/oran/dmaapadapter/configuration/ApplicationConfig.java b/src/main/java/org/oran/dmaapadapter/configuration/ApplicationConfig.java index 8c8e995..6beee21 100644 --- a/src/main/java/org/oran/dmaapadapter/configuration/ApplicationConfig.java +++ b/src/main/java/org/oran/dmaapadapter/configuration/ApplicationConfig.java @@ -96,10 +96,26 @@ public class ApplicationConfig { @Value("${app.kafka.bootstrap-servers:}") private String kafkaBootStrapServers; + @Getter + @Value("${app.kafka.max-poll-records:100}") + private int kafkaMaxPollRecords; + @Getter @Value("${app.pm-files-path:}") private String pmFilesPath; + @Getter + @Value("${app.s3.endpointOverride:}") + private String s3EndpointOverride; + + @Getter + @Value("${app.s3.accessKeyId:}") + private String s3AccessKeyId; + + @Getter + @Value("${app.s3.secretAccessKey:}") + private String s3SecretAccessKey; + private WebClientConfig webClientConfig = null; public WebClientConfig getWebClientConfig() { @@ -123,6 +139,10 @@ public class ApplicationConfig { return this.webClientConfig; } + public boolean isS3Enabled() { + return !s3EndpointOverride.isEmpty(); + } + // Adapter to parse the json format of the configuration file. static class ConfigFile { Collection types; diff --git a/src/main/java/org/oran/dmaapadapter/tasks/KafkaTopicListener.java b/src/main/java/org/oran/dmaapadapter/tasks/KafkaTopicListener.java index 61b50c6..8647b9b 100644 --- a/src/main/java/org/oran/dmaapadapter/tasks/KafkaTopicListener.java +++ b/src/main/java/org/oran/dmaapadapter/tasks/KafkaTopicListener.java @@ -23,12 +23,14 @@ package org.oran.dmaapadapter.tasks; import com.google.gson.Gson; import com.google.gson.GsonBuilder; +import java.net.URI; import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.Path; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.CompletableFuture; import lombok.Builder; import lombok.Getter; @@ -42,8 +44,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; import reactor.kafka.receiver.KafkaReceiver; import reactor.kafka.receiver.ReceiverOptions; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.core.ResponseBytes; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; /** * The class streams incoming requests from a Kafka topic and sends them further @@ -56,6 +68,9 @@ public class KafkaTopicListener implements TopicListener { private final InfoType type; private Flux dataFromTopic; + @Getter + private static S3AsyncClient s3AsynchClient; + private static Gson gson = new GsonBuilder() // .disableHtmlEscaping() // .create(); // @@ -65,11 +80,21 @@ public class KafkaTopicListener implements TopicListener { public static class NewFileEvent { @Getter private String filename; + + @Getter + private String objectStoreBucket; } public KafkaTopicListener(ApplicationConfig applicationConfig, InfoType type) { this.applicationConfig = applicationConfig; this.type = type; + if (applicationConfig.isS3Enabled()) { + synchronized (KafkaTopicListener.class) { + if (s3AsynchClient == null) { + s3AsynchClient = getS3AsyncClientBuilder().build(); + } + } + } } @Override @@ -92,26 +117,70 @@ public class KafkaTopicListener implements TopicListener { .doFinally(sig -> logger.error("KafkaTopicReceiver stopped, reason: {}", sig)) // .filter(t -> !t.value().isEmpty() || !t.key().isEmpty()) // .map(input -> new DataFromTopic(input.key(), input.value())) // - .map(this::getDataFromFileIfNewPmFileEvent) // + .flatMap(this::getDataFromFileIfNewPmFileEvent, 100) // .publish() // .autoConnect(1); } - private DataFromTopic getDataFromFileIfNewPmFileEvent(DataFromTopic data) { - - if (!applicationConfig.getPmFilesPath().isEmpty() // - && this.type.getDataType() == InfoType.DataType.PM_DATA // - && data.value.length() < 1000) { - try { - NewFileEvent ev = gson.fromJson(data.value, NewFileEvent.class); - Path path = Path.of(this.applicationConfig.getPmFilesPath(), ev.getFilename()); - String pmReportJson = Files.readString(path, Charset.defaultCharset()); - return new DataFromTopic(data.key, pmReportJson); - } catch (Exception e) { - return data; + private S3AsyncClientBuilder getS3AsyncClientBuilder() { + URI uri = URI.create(this.applicationConfig.getS3EndpointOverride()); + return S3AsyncClient.builder() // + .region(Region.US_EAST_1) // + .endpointOverride(uri) // + .credentialsProvider(StaticCredentialsProvider.create( // + AwsBasicCredentials.create(this.applicationConfig.getS3AccessKeyId(), // + this.applicationConfig.getS3SecretAccessKey()))); + + } + + private Mono getDataFromS3Object(String bucket, String key) { + if (!this.applicationConfig.isS3Enabled()) { + logger.error("Missing S3 confinguration in application.yaml, ignoring bucket: {}, key: {}", bucket, key); + return Mono.empty(); + } + + GetObjectRequest request = GetObjectRequest.builder() // + .bucket(bucket) // + .key(key) // + .build(); + + CompletableFuture> future = s3AsynchClient.getObject(request, + AsyncResponseTransformer.toBytes()); + + return Mono.fromFuture(future) // + .map(b -> new String(b.asByteArray(), Charset.defaultCharset())) // + .doOnError(t -> logger.error("Failed to get file from S3 {}", t.getMessage())) // + .doOnEach(n -> logger.debug("Read file from S3: {} {}", bucket, key)) // + .onErrorResume(t -> Mono.empty()); + } + + private Mono getDataFromFileIfNewPmFileEvent(DataFromTopic data) { + if (this.type.getDataType() != InfoType.DataType.PM_DATA || data.value.length() > 1000) { + return Mono.just(data); + } + + try { + NewFileEvent ev = gson.fromJson(data.value, NewFileEvent.class); + if (ev.getObjectStoreBucket() != null) { + if (applicationConfig.isS3Enabled()) { + return getDataFromS3Object(ev.getObjectStoreBucket(), ev.getFilename()) // + .map(str -> new DataFromTopic(data.key, str)); + } else { + logger.error("S3 is not configured in application.yaml, ignoring: {}", data); + return Mono.empty(); + } + } else { + if (applicationConfig.getPmFilesPath().isEmpty() || ev.filename == null) { + logger.debug("Passing data {}", data); + return Mono.just(data); + } else { + Path path = Path.of(this.applicationConfig.getPmFilesPath(), ev.getFilename()); + String pmReportJson = Files.readString(path, Charset.defaultCharset()); + return Mono.just(new DataFromTopic(data.key, pmReportJson)); + } } - } else { - return data; + } catch (Exception e) { + return Mono.just(data); } } @@ -126,6 +195,7 @@ public class KafkaTopicListener implements TopicListener { consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true); consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId); + consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, this.applicationConfig.getKafkaMaxPollRecords()); return ReceiverOptions.create(consumerProps) .subscription(Collections.singleton(this.type.getKafkaInputTopic())); diff --git a/src/test/java/org/oran/dmaapadapter/ApplicationTest.java b/src/test/java/org/oran/dmaapadapter/ApplicationTest.java index 2f1f6c9..7b6a3bc 100644 --- a/src/test/java/org/oran/dmaapadapter/ApplicationTest.java +++ b/src/test/java/org/oran/dmaapadapter/ApplicationTest.java @@ -25,14 +25,20 @@ import static org.awaitility.Awaitility.await; import static org.junit.jupiter.api.Assertions.assertTrue; import com.google.gson.JsonParser; +import com.google.protobuf.AbstractMessage.Builder; +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import com.google.protobuf.util.JsonFormat; import java.io.FileOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.lang.reflect.InvocationTargetException; import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.time.Instant; import java.util.ArrayList; import java.util.Map; @@ -59,6 +65,8 @@ import org.oran.dmaapadapter.tasks.JobDataDistributor; import org.oran.dmaapadapter.tasks.ProducerRegstrationTask; import org.oran.dmaapadapter.tasks.TopicListener; import org.oran.dmaapadapter.tasks.TopicListeners; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.test.context.SpringBootTest; import org.springframework.boot.test.context.SpringBootTest.WebEnvironment; @@ -115,6 +123,82 @@ class ApplicationTest { @LocalServerPort int localServerHttpPort; + private final Logger logger = LoggerFactory.getLogger(ApplicationTest.class); + + public static class ProtoJsonUtil { + + /** + * Makes a Json from a given message or builder + * + * @param messageOrBuilder is the instance + * @return The string representation + * @throws IOException if any error occurs + */ + public static String toJson(MessageOrBuilder messageOrBuilder) throws IOException { + return JsonFormat.printer().print(messageOrBuilder); + } + + /** + * Makes a new instance of message based on the json and the class + * + * @param is the class type + * @param json is the json instance + * @param clazz is the class instance + * @return An instance of T based on the json values + * @throws IOException if any error occurs + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static T fromJson(String json, Class clazz) throws IOException { + // https://stackoverflow.com/questions/27642021/calling-parsefrom-method-for-generic-protobuffer-class-in-java/33701202#33701202 + Builder builder = null; + try { + // Since we are dealing with a Message type, we can call newBuilder() + builder = (Builder) clazz.getMethod("newBuilder").invoke(null); + + } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException + | NoSuchMethodException | SecurityException e) { + return null; + } + + // The instance is placed into the builder values + JsonFormat.parser().ignoringUnknownFields().merge(json, builder); + + // the instance will be from the build + return (T) builder.build(); + } + } + + @Test + void testProtoBuf() throws Exception { + String path = "./src/test/resources/A20000626.2315+0200-2330+0200_HTTPS-6-73.xml.gz101.json"; + + String pmReportJson = Files.readString(Path.of(path), Charset.defaultCharset()); + + PmProtoGenerated.PmRopFile proto = ProtoJsonUtil.fromJson(pmReportJson, PmProtoGenerated.PmRopFile.class); + byte[] bytes = proto.toByteArray(); + + int TIMES = 100000; + { + Instant startTime = Instant.now(); + for (int i = 0; i < TIMES; ++i) { + PmProtoGenerated.PmRopFile.parseFrom(bytes); + } + long durationSeconds = Instant.now().getEpochSecond() - startTime.getEpochSecond(); + logger.info("*** Duration PROTO :" + durationSeconds + ", objects/second: " + TIMES / durationSeconds + + " time: " + (float) durationSeconds / TIMES); + } + { + Instant startTime = Instant.now(); + for (int i = 0; i < TIMES; ++i) { + PmReport reportsParsed = gson.fromJson(pmReportJson, PmReport.class); + } + long durationSeconds = Instant.now().getEpochSecond() - startTime.getEpochSecond(); + logger.info("*** Duration GSON :" + durationSeconds + ", objects/second: " + TIMES / durationSeconds + + " time: " + (float) durationSeconds / TIMES); + } + + } + static class TestApplicationConfig extends ApplicationConfig { @Override public String getIcsBaseUrl() { diff --git a/src/test/java/org/oran/dmaapadapter/IntegrationWithKafka.java b/src/test/java/org/oran/dmaapadapter/IntegrationWithKafka.java index 45bee5d..ba39e88 100644 --- a/src/test/java/org/oran/dmaapadapter/IntegrationWithKafka.java +++ b/src/test/java/org/oran/dmaapadapter/IntegrationWithKafka.java @@ -26,14 +26,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import com.google.gson.JsonParser; -import java.nio.charset.Charset; -import java.nio.file.Files; import java.nio.file.Path; import java.time.Duration; import java.time.Instant; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -71,9 +70,15 @@ import org.springframework.context.annotation.Bean; import org.springframework.test.context.TestPropertySource; import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; import reactor.kafka.sender.KafkaSender; import reactor.kafka.sender.SenderOptions; import reactor.kafka.sender.SenderRecord; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.CreateBucketResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; @SuppressWarnings("java:S3577") // Rename class @SpringBootTest(webEnvironment = WebEnvironment.DEFINED_PORT) @@ -169,8 +174,9 @@ class IntegrationWithKafka { // Create a listener to the output topic. The KafkaTopicListener happens to be // suitable for that, - InfoType type = - InfoType.builder().id("TestReceiver").kafkaInputTopic(OUTPUT_TOPIC).dataType("dataType").build(); + InfoType type = InfoType.builder() // + .id("TestReceiver_" + outputTopic) // + .kafkaInputTopic(OUTPUT_TOPIC).dataType("dataType").build(); KafkaTopicListener topicListener = new KafkaTopicListener(applicationConfig, type); @@ -304,7 +310,7 @@ class IntegrationWithKafka { } } - private SenderOptions senderOptions() { + private SenderOptions kafkaSenderOptions() { String bootstrapServers = this.applicationConfig.getKafkaBootStrapServers(); Map props = new HashMap<>(); @@ -316,14 +322,14 @@ class IntegrationWithKafka { return SenderOptions.create(props); } - private SenderRecord senderRecord(String data, String key, String typeId) { + private SenderRecord kafkaSenderRecord(String data, String key, String typeId) { final InfoType infoType = this.types.get(typeId); int correlationMetadata = 2; return SenderRecord.create(new ProducerRecord<>(infoType.getKafkaInputTopic(), key, data), correlationMetadata); } - private void sendDataToStream(Flux> dataToSend) { - final KafkaSender sender = KafkaSender.create(senderOptions()); + private void sendDataToKafka(Flux> dataToSend) { + final KafkaSender sender = KafkaSender.create(kafkaSenderOptions()); sender.send(dataToSend) // .doOnError(e -> logger.error("Send failed", e)) // @@ -355,6 +361,37 @@ class IntegrationWithKafka { Thread.sleep(4000); } + private Mono copyFileToS3Bucket(Path fileName, String s3Bucket, String s3Key) { + + PutObjectRequest request = PutObjectRequest.builder() // + .bucket(s3Bucket) // + .key(s3Key) // + .build(); + + AsyncRequestBody body = AsyncRequestBody.fromFile(fileName); + + CompletableFuture future = KafkaTopicListener.getS3AsynchClient().putObject(request, body); + + return Mono.fromFuture(future) // + .map(f -> s3Key) // + .doOnError(t -> logger.error("Failed to store file in S3 {}", t.getMessage())) + .onErrorResume(t -> Mono.empty()); + } + + private Mono createS3Bucket(String s3Bucket) { + + CreateBucketRequest request = CreateBucketRequest.builder() // + .bucket(s3Bucket) // + .build(); + + CompletableFuture future = KafkaTopicListener.getS3AsynchClient().createBucket(request); + + return Mono.fromFuture(future) // + .map(f -> s3Bucket) // + .doOnError(t -> logger.debug("Could not create S3 bucket: {}", t.getMessage())) + .onErrorResume(t -> Mono.just(s3Bucket)); + } + @Test void simpleCase() throws Exception { final String JOB_ID = "ID"; @@ -367,8 +404,8 @@ class IntegrationWithKafka { await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(1)); waitForKafkaListener(); - var dataToSend = Flux.just(senderRecord("Message", "", TYPE_ID)); - sendDataToStream(dataToSend); + var dataToSend = Flux.just(kafkaSenderRecord("Message", "", TYPE_ID)); + sendDataToKafka(dataToSend); verifiedReceivedByConsumer("Message"); } @@ -389,10 +426,10 @@ class IntegrationWithKafka { await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(2)); waitForKafkaListener(); - var dataToSend = Flux.range(1, 3).map(i -> senderRecord("Message_" + i, "", TYPE_ID)); // Message_1, - // Message_2 - // etc. - sendDataToStream(dataToSend); + var dataToSend = Flux.range(1, 3).map(i -> kafkaSenderRecord("Message_" + i, "", TYPE_ID)); // Message_1, + // Message_2 + // etc. + sendDataToKafka(dataToSend); verifiedReceivedByConsumer("Message_1", "[\"Message_1\", \"Message_2\", \"Message_3\"]"); } @@ -411,8 +448,8 @@ class IntegrationWithKafka { String sendString = "testData " + Instant.now(); String sendKey = "key " + Instant.now(); - var dataToSend = Flux.just(senderRecord(sendString, sendKey, TYPE_ID)); - sendDataToStream(dataToSend); + var dataToSend = Flux.just(kafkaSenderRecord(sendString, sendKey, TYPE_ID)); + sendDataToKafka(dataToSend); await().untilAsserted(() -> assertThat(kafkaReceiver.lastValue()).isEqualTo(sendString)); assertThat(kafkaReceiver.lastKey()).isEqualTo(sendKey); @@ -443,9 +480,9 @@ class IntegrationWithKafka { Instant startTime = Instant.now(); - var dataToSend = Flux.range(1, NO_OF_OBJECTS).map(i -> senderRecord("Message_" + i, "", TYPE_ID)); // Message_1, - // etc. - sendDataToStream(dataToSend); + var dataToSend = Flux.range(1, NO_OF_OBJECTS).map(i -> kafkaSenderRecord("Message_" + i, "", TYPE_ID)); // Message_1, + // etc. + sendDataToKafka(dataToSend); while (!kafkaReceiver.lastValue().equals("Message_" + NO_OF_OBJECTS)) { logger.info("sleeping {}", kafkaReceiver.lastValue()); @@ -458,7 +495,7 @@ class IntegrationWithKafka { @SuppressWarnings("squid:S2925") // "Thread.sleep" should not be used in tests. @Test - void kafkaCharacteristics_pmFilter() throws Exception { + void kafkaCharacteristics_pmFilter_localFile() throws Exception { // Filter PM reports and sent to two jobs over Kafka final String JOB_ID = "kafkaCharacteristics"; @@ -488,18 +525,65 @@ class IntegrationWithKafka { KafkaTopicListener.NewFileEvent.builder().filename("pm_report.json").build(); String eventAsString = gson.toJson(event); - String path = "./src/test/resources/pm_report.json"; - String pmReportJson = Files.readString(Path.of(path), Charset.defaultCharset()); - - var dataToSend = Flux.range(1, NO_OF_OBJECTS).map(i -> senderRecord(eventAsString, "key", PM_TYPE_ID)); - sendDataToStream(dataToSend); + var dataToSend = Flux.range(1, NO_OF_OBJECTS).map(i -> kafkaSenderRecord(eventAsString, "key", PM_TYPE_ID)); + sendDataToKafka(dataToSend); while (kafkaReceiver.count != NO_OF_OBJECTS) { logger.info("sleeping {}", kafkaReceiver.count); Thread.sleep(1000 * 1); } - // System.out.println(kafkaReceiver.receivedKafkaOutput.value); + final long durationSeconds = Instant.now().getEpochSecond() - startTime.getEpochSecond(); + logger.info("*** Duration :" + durationSeconds + ", objects/second: " + NO_OF_OBJECTS / durationSeconds); + logger.info("*** kafkaReceiver2 :" + kafkaReceiver.count); + + printStatistics(); + } + + @SuppressWarnings("squid:S2925") // "Thread.sleep" should not be used in tests. + @Test + void kafkaCharacteristics_pmFilter_s3() throws Exception { + // Filter PM reports and sent to two jobs over Kafka + + final String JOB_ID = "kafkaCharacteristics"; + final String JOB_ID2 = "kafkaCharacteristics2"; + + // Register producer, Register types + await().untilAsserted(() -> assertThat(icsSimulatorController.testResults.registrationInfo).isNotNull()); + assertThat(icsSimulatorController.testResults.registrationInfo.supportedTypeIds).hasSize(this.types.size()); + + PmReportFilter.FilterData filterData = new PmReportFilter.FilterData(); + filterData.getMeasTypes().add("succImmediateAssignProcs"); + filterData.getMeasObjClass().add("UtranCell"); + + this.icsSimulatorController.addJob(consumerJobInfoKafka(kafkaReceiver.OUTPUT_TOPIC, filterData), JOB_ID, + restClient()); + this.icsSimulatorController.addJob(consumerJobInfoKafka(kafkaReceiver2.OUTPUT_TOPIC, filterData), JOB_ID2, + restClient()); + + await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(2)); + waitForKafkaListener(); + + final int NO_OF_OBJECTS = 100000; + + Instant startTime = Instant.now(); + + KafkaTopicListener.NewFileEvent event = KafkaTopicListener.NewFileEvent.builder() // + .filename("pm_report.json").objectStoreBucket("ropfiles") // + .build(); + + createS3Bucket("ropfiles").block(); + copyFileToS3Bucket(Path.of("./src/test/resources/pm_report.json"), "ropfiles", "pm_report.json").block(); + + String eventAsString = gson.toJson(event); + + var dataToSend = Flux.range(1, NO_OF_OBJECTS).map(i -> kafkaSenderRecord(eventAsString, "key", PM_TYPE_ID)); + sendDataToKafka(dataToSend); + + while (kafkaReceiver.count != NO_OF_OBJECTS) { + logger.info("sleeping {}", kafkaReceiver.count); + Thread.sleep(1000 * 1); + } final long durationSeconds = Instant.now().getEpochSecond() - startTime.getEpochSecond(); logger.info("*** Duration :" + durationSeconds + ", objects/second: " + NO_OF_OBJECTS / durationSeconds); @@ -524,10 +608,10 @@ class IntegrationWithKafka { await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(2)); - var dataToSend = Flux.range(1, 100).map(i -> senderRecord("Message_" + i, "", TYPE_ID)); // Message_1, - // Message_2 - // etc. - sendDataToStream(dataToSend); // this should not overflow + var dataToSend = Flux.range(1, 100).map(i -> kafkaSenderRecord("Message_" + i, "", TYPE_ID)); // Message_1, + // Message_2 + // etc. + sendDataToKafka(dataToSend); // this should not overflow // Delete jobs, recreate one this.icsSimulatorController.deleteJob(JOB_ID1, restClient()); @@ -536,8 +620,8 @@ class IntegrationWithKafka { this.icsSimulatorController.addJob(consumerJobInfo(null, Duration.ZERO, 0, 1), JOB_ID2, restClient()); await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(1)); - dataToSend = Flux.just(senderRecord("Howdy", "", TYPE_ID)); - sendDataToStream(dataToSend); + dataToSend = Flux.just(kafkaSenderRecord("Howdy", "", TYPE_ID)); + sendDataToKafka(dataToSend); verifiedReceivedByConsumerLast("Howdy"); } diff --git a/src/test/java/org/oran/dmaapadapter/PmProtoGenerated.java b/src/test/java/org/oran/dmaapadapter/PmProtoGenerated.java new file mode 100644 index 0000000..f4fdcc8 --- /dev/null +++ b/src/test/java/org/oran/dmaapadapter/PmProtoGenerated.java @@ -0,0 +1,9383 @@ +/*- + * ========================LICENSE_START================================= + * O-RAN-SC + * %% + * Copyright (C) 2022 Nordix Foundation + * %% + * 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. + * ========================LICENSE_END=================================== + */ + +package org.oran.dmaapadapter; + +// Generated by the protocol buffer compiler. DO NOT EDIT! + +// source: pm_schema.proto + + +public final class PmProtoGenerated { + private PmProtoGenerated() {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + public interface PmRopFileOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile) + com.google.protobuf.MessageOrBuilder { + + /** + * .PmRopFile.Event event = 1; + */ + boolean hasEvent(); + + /** + * .PmRopFile.Event event = 1; + */ + PmProtoGenerated.PmRopFile.Event getEvent(); + + /** + * .PmRopFile.Event event = 1; + */ + PmProtoGenerated.PmRopFile.EventOrBuilder getEventOrBuilder(); + } + + /** + * Protobuf type {@code PmRopFile} + */ + public static final class PmRopFile extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile) + PmRopFileOrBuilder { + private static final long serialVersionUID = 0L; + + // Use PmRopFile.newBuilder() to construct. + private PmRopFile(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private PmRopFile() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private PmRopFile(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + PmProtoGenerated.PmRopFile.Event.Builder subBuilder = null; + if (event_ != null) { + subBuilder = event_.toBuilder(); + } + event_ = input.readMessage(PmProtoGenerated.PmRopFile.Event.parser(), extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(event_); + event_ = subBuilder.buildPartial(); + } + + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_fieldAccessorTable.ensureFieldAccessorsInitialized( + PmProtoGenerated.PmRopFile.class, PmProtoGenerated.PmRopFile.Builder.class); + } + + public interface CommoneventheaderOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Commoneventheader) + com.google.protobuf.MessageOrBuilder { + + /** + * string domain = 1; + */ + java.lang.String getDomain(); + + /** + * string domain = 1; + */ + com.google.protobuf.ByteString getDomainBytes(); + + /** + * string eventId = 2; + */ + java.lang.String getEventId(); + + /** + * string eventId = 2; + */ + com.google.protobuf.ByteString getEventIdBytes(); + + /** + * uint32 sequence = 3; + */ + int getSequence(); + + /** + * string eventName = 4; + */ + java.lang.String getEventName(); + + /** + * string eventName = 4; + */ + com.google.protobuf.ByteString getEventNameBytes(); + + /** + * string sourceName = 5; + */ + java.lang.String getSourceName(); + + /** + * string sourceName = 5; + */ + com.google.protobuf.ByteString getSourceNameBytes(); + + /** + * string reportingEntityName = 6; + */ + java.lang.String getReportingEntityName(); + + /** + * string reportingEntityName = 6; + */ + com.google.protobuf.ByteString getReportingEntityNameBytes(); + + /** + * string priority = 7; + */ + java.lang.String getPriority(); + + /** + * string priority = 7; + */ + com.google.protobuf.ByteString getPriorityBytes(); + + /** + * uint64 startEpochMicrosec = 8; + */ + long getStartEpochMicrosec(); + + /** + * uint64 lastEpochMicrosec = 9; + */ + long getLastEpochMicrosec(); + + /** + * string version = 10; + */ + java.lang.String getVersion(); + + /** + * string version = 10; + */ + com.google.protobuf.ByteString getVersionBytes(); + + /** + * string vesEventListenerVersion = 11; + */ + java.lang.String getVesEventListenerVersion(); + + /** + * string vesEventListenerVersion = 11; + */ + com.google.protobuf.ByteString getVesEventListenerVersionBytes(); + + /** + * string timeZoneOffset = 12; + */ + java.lang.String getTimeZoneOffset(); + + /** + * string timeZoneOffset = 12; + */ + com.google.protobuf.ByteString getTimeZoneOffsetBytes(); + } + + /** + * Protobuf type {@code PmRopFile.Commoneventheader} + */ + public static final class Commoneventheader extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Commoneventheader) + CommoneventheaderOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Commoneventheader.newBuilder() to construct. + private Commoneventheader(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Commoneventheader() { + domain_ = ""; + eventId_ = ""; + sequence_ = 0; + eventName_ = ""; + sourceName_ = ""; + reportingEntityName_ = ""; + priority_ = ""; + startEpochMicrosec_ = 0L; + lastEpochMicrosec_ = 0L; + version_ = ""; + vesEventListenerVersion_ = ""; + timeZoneOffset_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Commoneventheader(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + + domain_ = s; + break; + } + case 18: { + java.lang.String s = input.readStringRequireUtf8(); + + eventId_ = s; + break; + } + case 24: { + + sequence_ = input.readUInt32(); + break; + } + case 34: { + java.lang.String s = input.readStringRequireUtf8(); + + eventName_ = s; + break; + } + case 42: { + java.lang.String s = input.readStringRequireUtf8(); + + sourceName_ = s; + break; + } + case 50: { + java.lang.String s = input.readStringRequireUtf8(); + + reportingEntityName_ = s; + break; + } + case 58: { + java.lang.String s = input.readStringRequireUtf8(); + + priority_ = s; + break; + } + case 64: { + + startEpochMicrosec_ = input.readUInt64(); + break; + } + case 72: { + + lastEpochMicrosec_ = input.readUInt64(); + break; + } + case 82: { + java.lang.String s = input.readStringRequireUtf8(); + + version_ = s; + break; + } + case 90: { + java.lang.String s = input.readStringRequireUtf8(); + + vesEventListenerVersion_ = s; + break; + } + case 98: { + java.lang.String s = input.readStringRequireUtf8(); + + timeZoneOffset_ = s; + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Commoneventheader_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Commoneventheader_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Commoneventheader.class, + PmProtoGenerated.PmRopFile.Commoneventheader.Builder.class); + } + + public static final int DOMAIN_FIELD_NUMBER = 1; + private volatile java.lang.Object domain_; + + /** + * string domain = 1; + */ + public java.lang.String getDomain() { + java.lang.Object ref = domain_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + domain_ = s; + return s; + } + } + + /** + * string domain = 1; + */ + public com.google.protobuf.ByteString getDomainBytes() { + java.lang.Object ref = domain_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + domain_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EVENTID_FIELD_NUMBER = 2; + private volatile java.lang.Object eventId_; + + /** + * string eventId = 2; + */ + public java.lang.String getEventId() { + java.lang.Object ref = eventId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + eventId_ = s; + return s; + } + } + + /** + * string eventId = 2; + */ + public com.google.protobuf.ByteString getEventIdBytes() { + java.lang.Object ref = eventId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + eventId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SEQUENCE_FIELD_NUMBER = 3; + private int sequence_; + + /** + * uint32 sequence = 3; + */ + public int getSequence() { + return sequence_; + } + + public static final int EVENTNAME_FIELD_NUMBER = 4; + private volatile java.lang.Object eventName_; + + /** + * string eventName = 4; + */ + public java.lang.String getEventName() { + java.lang.Object ref = eventName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + eventName_ = s; + return s; + } + } + + /** + * string eventName = 4; + */ + public com.google.protobuf.ByteString getEventNameBytes() { + java.lang.Object ref = eventName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + eventName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SOURCENAME_FIELD_NUMBER = 5; + private volatile java.lang.Object sourceName_; + + /** + * string sourceName = 5; + */ + public java.lang.String getSourceName() { + java.lang.Object ref = sourceName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sourceName_ = s; + return s; + } + } + + /** + * string sourceName = 5; + */ + public com.google.protobuf.ByteString getSourceNameBytes() { + java.lang.Object ref = sourceName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sourceName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int REPORTINGENTITYNAME_FIELD_NUMBER = 6; + private volatile java.lang.Object reportingEntityName_; + + /** + * string reportingEntityName = 6; + */ + public java.lang.String getReportingEntityName() { + java.lang.Object ref = reportingEntityName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + reportingEntityName_ = s; + return s; + } + } + + /** + * string reportingEntityName = 6; + */ + public com.google.protobuf.ByteString getReportingEntityNameBytes() { + java.lang.Object ref = reportingEntityName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + reportingEntityName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PRIORITY_FIELD_NUMBER = 7; + private volatile java.lang.Object priority_; + + /** + * string priority = 7; + */ + public java.lang.String getPriority() { + java.lang.Object ref = priority_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + priority_ = s; + return s; + } + } + + /** + * string priority = 7; + */ + public com.google.protobuf.ByteString getPriorityBytes() { + java.lang.Object ref = priority_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + priority_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int STARTEPOCHMICROSEC_FIELD_NUMBER = 8; + private long startEpochMicrosec_; + + /** + * uint64 startEpochMicrosec = 8; + */ + public long getStartEpochMicrosec() { + return startEpochMicrosec_; + } + + public static final int LASTEPOCHMICROSEC_FIELD_NUMBER = 9; + private long lastEpochMicrosec_; + + /** + * uint64 lastEpochMicrosec = 9; + */ + public long getLastEpochMicrosec() { + return lastEpochMicrosec_; + } + + public static final int VERSION_FIELD_NUMBER = 10; + private volatile java.lang.Object version_; + + /** + * string version = 10; + */ + public java.lang.String getVersion() { + java.lang.Object ref = version_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + version_ = s; + return s; + } + } + + /** + * string version = 10; + */ + public com.google.protobuf.ByteString getVersionBytes() { + java.lang.Object ref = version_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + version_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int VESEVENTLISTENERVERSION_FIELD_NUMBER = 11; + private volatile java.lang.Object vesEventListenerVersion_; + + /** + * string vesEventListenerVersion = 11; + */ + public java.lang.String getVesEventListenerVersion() { + java.lang.Object ref = vesEventListenerVersion_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + vesEventListenerVersion_ = s; + return s; + } + } + + /** + * string vesEventListenerVersion = 11; + */ + public com.google.protobuf.ByteString getVesEventListenerVersionBytes() { + java.lang.Object ref = vesEventListenerVersion_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + vesEventListenerVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TIMEZONEOFFSET_FIELD_NUMBER = 12; + private volatile java.lang.Object timeZoneOffset_; + + /** + * string timeZoneOffset = 12; + */ + public java.lang.String getTimeZoneOffset() { + java.lang.Object ref = timeZoneOffset_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + timeZoneOffset_ = s; + return s; + } + } + + /** + * string timeZoneOffset = 12; + */ + public com.google.protobuf.ByteString getTimeZoneOffsetBytes() { + java.lang.Object ref = timeZoneOffset_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + timeZoneOffset_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getDomainBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, domain_); + } + if (!getEventIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, eventId_); + } + if (sequence_ != 0) { + output.writeUInt32(3, sequence_); + } + if (!getEventNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 4, eventName_); + } + if (!getSourceNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 5, sourceName_); + } + if (!getReportingEntityNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 6, reportingEntityName_); + } + if (!getPriorityBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 7, priority_); + } + if (startEpochMicrosec_ != 0L) { + output.writeUInt64(8, startEpochMicrosec_); + } + if (lastEpochMicrosec_ != 0L) { + output.writeUInt64(9, lastEpochMicrosec_); + } + if (!getVersionBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 10, version_); + } + if (!getVesEventListenerVersionBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 11, vesEventListenerVersion_); + } + if (!getTimeZoneOffsetBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 12, timeZoneOffset_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (!getDomainBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, domain_); + } + if (!getEventIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, eventId_); + } + if (sequence_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeUInt32Size(3, sequence_); + } + if (!getEventNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, eventName_); + } + if (!getSourceNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, sourceName_); + } + if (!getReportingEntityNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, reportingEntityName_); + } + if (!getPriorityBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(7, priority_); + } + if (startEpochMicrosec_ != 0L) { + size += com.google.protobuf.CodedOutputStream.computeUInt64Size(8, startEpochMicrosec_); + } + if (lastEpochMicrosec_ != 0L) { + size += com.google.protobuf.CodedOutputStream.computeUInt64Size(9, lastEpochMicrosec_); + } + if (!getVersionBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(10, version_); + } + if (!getVesEventListenerVersionBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(11, vesEventListenerVersion_); + } + if (!getTimeZoneOffsetBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(12, timeZoneOffset_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Commoneventheader)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Commoneventheader other = (PmProtoGenerated.PmRopFile.Commoneventheader) obj; + + boolean result = true; + result = result && getDomain().equals(other.getDomain()); + result = result && getEventId().equals(other.getEventId()); + result = result && (getSequence() == other.getSequence()); + result = result && getEventName().equals(other.getEventName()); + result = result && getSourceName().equals(other.getSourceName()); + result = result && getReportingEntityName().equals(other.getReportingEntityName()); + result = result && getPriority().equals(other.getPriority()); + result = result && (getStartEpochMicrosec() == other.getStartEpochMicrosec()); + result = result && (getLastEpochMicrosec() == other.getLastEpochMicrosec()); + result = result && getVersion().equals(other.getVersion()); + result = result && getVesEventListenerVersion().equals(other.getVesEventListenerVersion()); + result = result && getTimeZoneOffset().equals(other.getTimeZoneOffset()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + DOMAIN_FIELD_NUMBER; + hash = (53 * hash) + getDomain().hashCode(); + hash = (37 * hash) + EVENTID_FIELD_NUMBER; + hash = (53 * hash) + getEventId().hashCode(); + hash = (37 * hash) + SEQUENCE_FIELD_NUMBER; + hash = (53 * hash) + getSequence(); + hash = (37 * hash) + EVENTNAME_FIELD_NUMBER; + hash = (53 * hash) + getEventName().hashCode(); + hash = (37 * hash) + SOURCENAME_FIELD_NUMBER; + hash = (53 * hash) + getSourceName().hashCode(); + hash = (37 * hash) + REPORTINGENTITYNAME_FIELD_NUMBER; + hash = (53 * hash) + getReportingEntityName().hashCode(); + hash = (37 * hash) + PRIORITY_FIELD_NUMBER; + hash = (53 * hash) + getPriority().hashCode(); + hash = (37 * hash) + STARTEPOCHMICROSEC_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong(getStartEpochMicrosec()); + hash = (37 * hash) + LASTEPOCHMICROSEC_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong(getLastEpochMicrosec()); + hash = (37 * hash) + VERSION_FIELD_NUMBER; + hash = (53 * hash) + getVersion().hashCode(); + hash = (37 * hash) + VESEVENTLISTENERVERSION_FIELD_NUMBER; + hash = (53 * hash) + getVesEventListenerVersion().hashCode(); + hash = (37 * hash) + TIMEZONEOFFSET_FIELD_NUMBER; + hash = (53 * hash) + getTimeZoneOffset().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Commoneventheader prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Commoneventheader} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Commoneventheader) + PmProtoGenerated.PmRopFile.CommoneventheaderOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Commoneventheader_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Commoneventheader_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Commoneventheader.class, + PmProtoGenerated.PmRopFile.Commoneventheader.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Commoneventheader.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + domain_ = ""; + + eventId_ = ""; + + sequence_ = 0; + + eventName_ = ""; + + sourceName_ = ""; + + reportingEntityName_ = ""; + + priority_ = ""; + + startEpochMicrosec_ = 0L; + + lastEpochMicrosec_ = 0L; + + version_ = ""; + + vesEventListenerVersion_ = ""; + + timeZoneOffset_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Commoneventheader_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Commoneventheader getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Commoneventheader.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Commoneventheader build() { + PmProtoGenerated.PmRopFile.Commoneventheader result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Commoneventheader buildPartial() { + PmProtoGenerated.PmRopFile.Commoneventheader result = + new PmProtoGenerated.PmRopFile.Commoneventheader(this); + result.domain_ = domain_; + result.eventId_ = eventId_; + result.sequence_ = sequence_; + result.eventName_ = eventName_; + result.sourceName_ = sourceName_; + result.reportingEntityName_ = reportingEntityName_; + result.priority_ = priority_; + result.startEpochMicrosec_ = startEpochMicrosec_; + result.lastEpochMicrosec_ = lastEpochMicrosec_; + result.version_ = version_; + result.vesEventListenerVersion_ = vesEventListenerVersion_; + result.timeZoneOffset_ = timeZoneOffset_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Commoneventheader) { + return mergeFrom((PmProtoGenerated.PmRopFile.Commoneventheader) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Commoneventheader other) { + if (other == PmProtoGenerated.PmRopFile.Commoneventheader.getDefaultInstance()) + return this; + if (!other.getDomain().isEmpty()) { + domain_ = other.domain_; + onChanged(); + } + if (!other.getEventId().isEmpty()) { + eventId_ = other.eventId_; + onChanged(); + } + if (other.getSequence() != 0) { + setSequence(other.getSequence()); + } + if (!other.getEventName().isEmpty()) { + eventName_ = other.eventName_; + onChanged(); + } + if (!other.getSourceName().isEmpty()) { + sourceName_ = other.sourceName_; + onChanged(); + } + if (!other.getReportingEntityName().isEmpty()) { + reportingEntityName_ = other.reportingEntityName_; + onChanged(); + } + if (!other.getPriority().isEmpty()) { + priority_ = other.priority_; + onChanged(); + } + if (other.getStartEpochMicrosec() != 0L) { + setStartEpochMicrosec(other.getStartEpochMicrosec()); + } + if (other.getLastEpochMicrosec() != 0L) { + setLastEpochMicrosec(other.getLastEpochMicrosec()); + } + if (!other.getVersion().isEmpty()) { + version_ = other.version_; + onChanged(); + } + if (!other.getVesEventListenerVersion().isEmpty()) { + vesEventListenerVersion_ = other.vesEventListenerVersion_; + onChanged(); + } + if (!other.getTimeZoneOffset().isEmpty()) { + timeZoneOffset_ = other.timeZoneOffset_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Commoneventheader parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Commoneventheader) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object domain_ = ""; + + /** + * string domain = 1; + */ + public java.lang.String getDomain() { + java.lang.Object ref = domain_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + domain_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string domain = 1; + */ + public com.google.protobuf.ByteString getDomainBytes() { + java.lang.Object ref = domain_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + domain_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string domain = 1; + */ + public Builder setDomain(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + domain_ = value; + onChanged(); + return this; + } + + /** + * string domain = 1; + */ + public Builder clearDomain() { + + domain_ = getDefaultInstance().getDomain(); + onChanged(); + return this; + } + + /** + * string domain = 1; + */ + public Builder setDomainBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + domain_ = value; + onChanged(); + return this; + } + + private java.lang.Object eventId_ = ""; + + /** + * string eventId = 2; + */ + public java.lang.String getEventId() { + java.lang.Object ref = eventId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + eventId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string eventId = 2; + */ + public com.google.protobuf.ByteString getEventIdBytes() { + java.lang.Object ref = eventId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + eventId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string eventId = 2; + */ + public Builder setEventId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + eventId_ = value; + onChanged(); + return this; + } + + /** + * string eventId = 2; + */ + public Builder clearEventId() { + + eventId_ = getDefaultInstance().getEventId(); + onChanged(); + return this; + } + + /** + * string eventId = 2; + */ + public Builder setEventIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + eventId_ = value; + onChanged(); + return this; + } + + private int sequence_; + + /** + * uint32 sequence = 3; + */ + public int getSequence() { + return sequence_; + } + + /** + * uint32 sequence = 3; + */ + public Builder setSequence(int value) { + + sequence_ = value; + onChanged(); + return this; + } + + /** + * uint32 sequence = 3; + */ + public Builder clearSequence() { + + sequence_ = 0; + onChanged(); + return this; + } + + private java.lang.Object eventName_ = ""; + + /** + * string eventName = 4; + */ + public java.lang.String getEventName() { + java.lang.Object ref = eventName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + eventName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string eventName = 4; + */ + public com.google.protobuf.ByteString getEventNameBytes() { + java.lang.Object ref = eventName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + eventName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string eventName = 4; + */ + public Builder setEventName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + eventName_ = value; + onChanged(); + return this; + } + + /** + * string eventName = 4; + */ + public Builder clearEventName() { + + eventName_ = getDefaultInstance().getEventName(); + onChanged(); + return this; + } + + /** + * string eventName = 4; + */ + public Builder setEventNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + eventName_ = value; + onChanged(); + return this; + } + + private java.lang.Object sourceName_ = ""; + + /** + * string sourceName = 5; + */ + public java.lang.String getSourceName() { + java.lang.Object ref = sourceName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sourceName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string sourceName = 5; + */ + public com.google.protobuf.ByteString getSourceNameBytes() { + java.lang.Object ref = sourceName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sourceName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string sourceName = 5; + */ + public Builder setSourceName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + sourceName_ = value; + onChanged(); + return this; + } + + /** + * string sourceName = 5; + */ + public Builder clearSourceName() { + + sourceName_ = getDefaultInstance().getSourceName(); + onChanged(); + return this; + } + + /** + * string sourceName = 5; + */ + public Builder setSourceNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + sourceName_ = value; + onChanged(); + return this; + } + + private java.lang.Object reportingEntityName_ = ""; + + /** + * string reportingEntityName = 6; + */ + public java.lang.String getReportingEntityName() { + java.lang.Object ref = reportingEntityName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + reportingEntityName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string reportingEntityName = 6; + */ + public com.google.protobuf.ByteString getReportingEntityNameBytes() { + java.lang.Object ref = reportingEntityName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + reportingEntityName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string reportingEntityName = 6; + */ + public Builder setReportingEntityName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + reportingEntityName_ = value; + onChanged(); + return this; + } + + /** + * string reportingEntityName = 6; + */ + public Builder clearReportingEntityName() { + + reportingEntityName_ = getDefaultInstance().getReportingEntityName(); + onChanged(); + return this; + } + + /** + * string reportingEntityName = 6; + */ + public Builder setReportingEntityNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + reportingEntityName_ = value; + onChanged(); + return this; + } + + private java.lang.Object priority_ = ""; + + /** + * string priority = 7; + */ + public java.lang.String getPriority() { + java.lang.Object ref = priority_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + priority_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string priority = 7; + */ + public com.google.protobuf.ByteString getPriorityBytes() { + java.lang.Object ref = priority_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + priority_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string priority = 7; + */ + public Builder setPriority(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + priority_ = value; + onChanged(); + return this; + } + + /** + * string priority = 7; + */ + public Builder clearPriority() { + + priority_ = getDefaultInstance().getPriority(); + onChanged(); + return this; + } + + /** + * string priority = 7; + */ + public Builder setPriorityBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + priority_ = value; + onChanged(); + return this; + } + + private long startEpochMicrosec_; + + /** + * uint64 startEpochMicrosec = 8; + */ + public long getStartEpochMicrosec() { + return startEpochMicrosec_; + } + + /** + * uint64 startEpochMicrosec = 8; + */ + public Builder setStartEpochMicrosec(long value) { + + startEpochMicrosec_ = value; + onChanged(); + return this; + } + + /** + * uint64 startEpochMicrosec = 8; + */ + public Builder clearStartEpochMicrosec() { + + startEpochMicrosec_ = 0L; + onChanged(); + return this; + } + + private long lastEpochMicrosec_; + + /** + * uint64 lastEpochMicrosec = 9; + */ + public long getLastEpochMicrosec() { + return lastEpochMicrosec_; + } + + /** + * uint64 lastEpochMicrosec = 9; + */ + public Builder setLastEpochMicrosec(long value) { + + lastEpochMicrosec_ = value; + onChanged(); + return this; + } + + /** + * uint64 lastEpochMicrosec = 9; + */ + public Builder clearLastEpochMicrosec() { + + lastEpochMicrosec_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object version_ = ""; + + /** + * string version = 10; + */ + public java.lang.String getVersion() { + java.lang.Object ref = version_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + version_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string version = 10; + */ + public com.google.protobuf.ByteString getVersionBytes() { + java.lang.Object ref = version_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + version_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string version = 10; + */ + public Builder setVersion(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + version_ = value; + onChanged(); + return this; + } + + /** + * string version = 10; + */ + public Builder clearVersion() { + + version_ = getDefaultInstance().getVersion(); + onChanged(); + return this; + } + + /** + * string version = 10; + */ + public Builder setVersionBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + version_ = value; + onChanged(); + return this; + } + + private java.lang.Object vesEventListenerVersion_ = ""; + + /** + * string vesEventListenerVersion = 11; + */ + public java.lang.String getVesEventListenerVersion() { + java.lang.Object ref = vesEventListenerVersion_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + vesEventListenerVersion_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string vesEventListenerVersion = 11; + */ + public com.google.protobuf.ByteString getVesEventListenerVersionBytes() { + java.lang.Object ref = vesEventListenerVersion_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + vesEventListenerVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string vesEventListenerVersion = 11; + */ + public Builder setVesEventListenerVersion(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + vesEventListenerVersion_ = value; + onChanged(); + return this; + } + + /** + * string vesEventListenerVersion = 11; + */ + public Builder clearVesEventListenerVersion() { + + vesEventListenerVersion_ = getDefaultInstance().getVesEventListenerVersion(); + onChanged(); + return this; + } + + /** + * string vesEventListenerVersion = 11; + */ + public Builder setVesEventListenerVersionBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + vesEventListenerVersion_ = value; + onChanged(); + return this; + } + + private java.lang.Object timeZoneOffset_ = ""; + + /** + * string timeZoneOffset = 12; + */ + public java.lang.String getTimeZoneOffset() { + java.lang.Object ref = timeZoneOffset_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + timeZoneOffset_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string timeZoneOffset = 12; + */ + public com.google.protobuf.ByteString getTimeZoneOffsetBytes() { + java.lang.Object ref = timeZoneOffset_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + timeZoneOffset_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string timeZoneOffset = 12; + */ + public Builder setTimeZoneOffset(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + timeZoneOffset_ = value; + onChanged(); + return this; + } + + /** + * string timeZoneOffset = 12; + */ + public Builder clearTimeZoneOffset() { + + timeZoneOffset_ = getDefaultInstance().getTimeZoneOffset(); + onChanged(); + return this; + } + + /** + * string timeZoneOffset = 12; + */ + public Builder setTimeZoneOffsetBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + timeZoneOffset_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Commoneventheader) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Commoneventheader) + private static final PmProtoGenerated.PmRopFile.Commoneventheader DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Commoneventheader(); + } + + public static PmProtoGenerated.PmRopFile.Commoneventheader getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Commoneventheader parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Commoneventheader(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Commoneventheader getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeasinfoidOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Measinfoid) + com.google.protobuf.MessageOrBuilder { + + /** + * string sMeasInfoId = 1; + */ + java.lang.String getSMeasInfoId(); + + /** + * string sMeasInfoId = 1; + */ + com.google.protobuf.ByteString getSMeasInfoIdBytes(); + } + + /** + * Protobuf type {@code PmRopFile.Measinfoid} + */ + public static final class Measinfoid extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Measinfoid) + MeasinfoidOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Measinfoid.newBuilder() to construct. + private Measinfoid(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Measinfoid() { + sMeasInfoId_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Measinfoid(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + + sMeasInfoId_ = s; + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfoid_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfoid_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measinfoid.class, + PmProtoGenerated.PmRopFile.Measinfoid.Builder.class); + } + + public static final int SMEASINFOID_FIELD_NUMBER = 1; + private volatile java.lang.Object sMeasInfoId_; + + /** + * string sMeasInfoId = 1; + */ + public java.lang.String getSMeasInfoId() { + java.lang.Object ref = sMeasInfoId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sMeasInfoId_ = s; + return s; + } + } + + /** + * string sMeasInfoId = 1; + */ + public com.google.protobuf.ByteString getSMeasInfoIdBytes() { + java.lang.Object ref = sMeasInfoId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sMeasInfoId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getSMeasInfoIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, sMeasInfoId_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (!getSMeasInfoIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, sMeasInfoId_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Measinfoid)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Measinfoid other = (PmProtoGenerated.PmRopFile.Measinfoid) obj; + + boolean result = true; + result = result && getSMeasInfoId().equals(other.getSMeasInfoId()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + SMEASINFOID_FIELD_NUMBER; + hash = (53 * hash) + getSMeasInfoId().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Measinfoid prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Measinfoid} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Measinfoid) + PmProtoGenerated.PmRopFile.MeasinfoidOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfoid_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfoid_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measinfoid.class, + PmProtoGenerated.PmRopFile.Measinfoid.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Measinfoid.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + sMeasInfoId_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfoid_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfoid getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Measinfoid.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfoid build() { + PmProtoGenerated.PmRopFile.Measinfoid result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfoid buildPartial() { + PmProtoGenerated.PmRopFile.Measinfoid result = new PmProtoGenerated.PmRopFile.Measinfoid(this); + result.sMeasInfoId_ = sMeasInfoId_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Measinfoid) { + return mergeFrom((PmProtoGenerated.PmRopFile.Measinfoid) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Measinfoid other) { + if (other == PmProtoGenerated.PmRopFile.Measinfoid.getDefaultInstance()) + return this; + if (!other.getSMeasInfoId().isEmpty()) { + sMeasInfoId_ = other.sMeasInfoId_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Measinfoid parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Measinfoid) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object sMeasInfoId_ = ""; + + /** + * string sMeasInfoId = 1; + */ + public java.lang.String getSMeasInfoId() { + java.lang.Object ref = sMeasInfoId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sMeasInfoId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string sMeasInfoId = 1; + */ + public com.google.protobuf.ByteString getSMeasInfoIdBytes() { + java.lang.Object ref = sMeasInfoId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sMeasInfoId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string sMeasInfoId = 1; + */ + public Builder setSMeasInfoId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + sMeasInfoId_ = value; + onChanged(); + return this; + } + + /** + * string sMeasInfoId = 1; + */ + public Builder clearSMeasInfoId() { + + sMeasInfoId_ = getDefaultInstance().getSMeasInfoId(); + onChanged(); + return this; + } + + /** + * string sMeasInfoId = 1; + */ + public Builder setSMeasInfoIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + sMeasInfoId_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Measinfoid) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Measinfoid) + private static final PmProtoGenerated.PmRopFile.Measinfoid DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Measinfoid(); + } + + public static PmProtoGenerated.PmRopFile.Measinfoid getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Measinfoid parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Measinfoid(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfoid getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeastypesOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Meastypes) + com.google.protobuf.MessageOrBuilder { + + /** + * repeated string sMeasTypesList = 1; + */ + java.util.List getSMeasTypesListList(); + + /** + * repeated string sMeasTypesList = 1; + */ + int getSMeasTypesListCount(); + + /** + * repeated string sMeasTypesList = 1; + */ + java.lang.String getSMeasTypesList(int index); + + /** + * repeated string sMeasTypesList = 1; + */ + com.google.protobuf.ByteString getSMeasTypesListBytes(int index); + } + + /** + * Protobuf type {@code PmRopFile.Meastypes} + */ + public static final class Meastypes extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Meastypes) + MeastypesOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Meastypes.newBuilder() to construct. + private Meastypes(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Meastypes() { + sMeasTypesList_ = com.google.protobuf.LazyStringArrayList.EMPTY; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Meastypes(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + sMeasTypesList_ = new com.google.protobuf.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000001; + } + sMeasTypesList_.add(s); + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + sMeasTypesList_ = sMeasTypesList_.getUnmodifiableView(); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Meastypes_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Meastypes_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Meastypes.class, + PmProtoGenerated.PmRopFile.Meastypes.Builder.class); + } + + public static final int SMEASTYPESLIST_FIELD_NUMBER = 1; + private com.google.protobuf.LazyStringList sMeasTypesList_; + + /** + * repeated string sMeasTypesList = 1; + */ + public com.google.protobuf.ProtocolStringList getSMeasTypesListList() { + return sMeasTypesList_; + } + + /** + * repeated string sMeasTypesList = 1; + */ + public int getSMeasTypesListCount() { + return sMeasTypesList_.size(); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public java.lang.String getSMeasTypesList(int index) { + return sMeasTypesList_.get(index); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public com.google.protobuf.ByteString getSMeasTypesListBytes(int index) { + return sMeasTypesList_.getByteString(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + for (int i = 0; i < sMeasTypesList_.size(); i++) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, sMeasTypesList_.getRaw(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + { + int dataSize = 0; + for (int i = 0; i < sMeasTypesList_.size(); i++) { + dataSize += computeStringSizeNoTag(sMeasTypesList_.getRaw(i)); + } + size += dataSize; + size += 1 * getSMeasTypesListList().size(); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Meastypes)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Meastypes other = (PmProtoGenerated.PmRopFile.Meastypes) obj; + + boolean result = true; + result = result && getSMeasTypesListList().equals(other.getSMeasTypesListList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getSMeasTypesListCount() > 0) { + hash = (37 * hash) + SMEASTYPESLIST_FIELD_NUMBER; + hash = (53 * hash) + getSMeasTypesListList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Meastypes parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Meastypes prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Meastypes} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Meastypes) + PmProtoGenerated.PmRopFile.MeastypesOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Meastypes_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Meastypes_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Meastypes.class, + PmProtoGenerated.PmRopFile.Meastypes.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Meastypes.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + sMeasTypesList_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Meastypes_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Meastypes getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Meastypes.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Meastypes build() { + PmProtoGenerated.PmRopFile.Meastypes result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Meastypes buildPartial() { + PmProtoGenerated.PmRopFile.Meastypes result = new PmProtoGenerated.PmRopFile.Meastypes(this); + int from_bitField0_ = bitField0_; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + sMeasTypesList_ = sMeasTypesList_.getUnmodifiableView(); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.sMeasTypesList_ = sMeasTypesList_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Meastypes) { + return mergeFrom((PmProtoGenerated.PmRopFile.Meastypes) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Meastypes other) { + if (other == PmProtoGenerated.PmRopFile.Meastypes.getDefaultInstance()) + return this; + if (!other.sMeasTypesList_.isEmpty()) { + if (sMeasTypesList_.isEmpty()) { + sMeasTypesList_ = other.sMeasTypesList_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureSMeasTypesListIsMutable(); + sMeasTypesList_.addAll(other.sMeasTypesList_); + } + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Meastypes parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Meastypes) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private com.google.protobuf.LazyStringList sMeasTypesList_ = + com.google.protobuf.LazyStringArrayList.EMPTY; + + private void ensureSMeasTypesListIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + sMeasTypesList_ = new com.google.protobuf.LazyStringArrayList(sMeasTypesList_); + bitField0_ |= 0x00000001; + } + } + + /** + * repeated string sMeasTypesList = 1; + */ + public com.google.protobuf.ProtocolStringList getSMeasTypesListList() { + return sMeasTypesList_.getUnmodifiableView(); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public int getSMeasTypesListCount() { + return sMeasTypesList_.size(); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public java.lang.String getSMeasTypesList(int index) { + return sMeasTypesList_.get(index); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public com.google.protobuf.ByteString getSMeasTypesListBytes(int index) { + return sMeasTypesList_.getByteString(index); + } + + /** + * repeated string sMeasTypesList = 1; + */ + public Builder setSMeasTypesList(int index, java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureSMeasTypesListIsMutable(); + sMeasTypesList_.set(index, value); + onChanged(); + return this; + } + + /** + * repeated string sMeasTypesList = 1; + */ + public Builder addSMeasTypesList(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureSMeasTypesListIsMutable(); + sMeasTypesList_.add(value); + onChanged(); + return this; + } + + /** + * repeated string sMeasTypesList = 1; + */ + public Builder addAllSMeasTypesList(java.lang.Iterable values) { + ensureSMeasTypesListIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll(values, sMeasTypesList_); + onChanged(); + return this; + } + + /** + * repeated string sMeasTypesList = 1; + */ + public Builder clearSMeasTypesList() { + sMeasTypesList_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + + /** + * repeated string sMeasTypesList = 1; + */ + public Builder addSMeasTypesListBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + ensureSMeasTypesListIsMutable(); + sMeasTypesList_.add(value); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Meastypes) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Meastypes) + private static final PmProtoGenerated.PmRopFile.Meastypes DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Meastypes(); + } + + public static PmProtoGenerated.PmRopFile.Meastypes getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Meastypes parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Meastypes(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Meastypes getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeasresultsOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Measresults) + com.google.protobuf.MessageOrBuilder { + + /** + * uint32 p = 1; + */ + int getP(); + + /** + * string sValue = 2; + */ + java.lang.String getSValue(); + + /** + * string sValue = 2; + */ + com.google.protobuf.ByteString getSValueBytes(); + } + + /** + * Protobuf type {@code PmRopFile.Measresults} + */ + public static final class Measresults extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Measresults) + MeasresultsOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Measresults.newBuilder() to construct. + private Measresults(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Measresults() { + p_ = 0; + sValue_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Measresults(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + + p_ = input.readUInt32(); + break; + } + case 18: { + java.lang.String s = input.readStringRequireUtf8(); + + sValue_ = s; + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measresults_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measresults_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measresults.class, + PmProtoGenerated.PmRopFile.Measresults.Builder.class); + } + + public static final int P_FIELD_NUMBER = 1; + private int p_; + + /** + * uint32 p = 1; + */ + public int getP() { + return p_; + } + + public static final int SVALUE_FIELD_NUMBER = 2; + private volatile java.lang.Object sValue_; + + /** + * string sValue = 2; + */ + public java.lang.String getSValue() { + java.lang.Object ref = sValue_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sValue_ = s; + return s; + } + } + + /** + * string sValue = 2; + */ + public com.google.protobuf.ByteString getSValueBytes() { + java.lang.Object ref = sValue_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sValue_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (p_ != 0) { + output.writeUInt32(1, p_); + } + if (!getSValueBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, sValue_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (p_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeUInt32Size(1, p_); + } + if (!getSValueBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, sValue_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Measresults)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Measresults other = (PmProtoGenerated.PmRopFile.Measresults) obj; + + boolean result = true; + result = result && (getP() == other.getP()); + result = result && getSValue().equals(other.getSValue()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + P_FIELD_NUMBER; + hash = (53 * hash) + getP(); + hash = (37 * hash) + SVALUE_FIELD_NUMBER; + hash = (53 * hash) + getSValue().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measresults parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Measresults prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Measresults} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Measresults) + PmProtoGenerated.PmRopFile.MeasresultsOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measresults_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measresults_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measresults.class, + PmProtoGenerated.PmRopFile.Measresults.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Measresults.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + p_ = 0; + + sValue_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Measresults_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measresults getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Measresults.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measresults build() { + PmProtoGenerated.PmRopFile.Measresults result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measresults buildPartial() { + PmProtoGenerated.PmRopFile.Measresults result = new PmProtoGenerated.PmRopFile.Measresults(this); + result.p_ = p_; + result.sValue_ = sValue_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Measresults) { + return mergeFrom((PmProtoGenerated.PmRopFile.Measresults) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Measresults other) { + if (other == PmProtoGenerated.PmRopFile.Measresults.getDefaultInstance()) + return this; + if (other.getP() != 0) { + setP(other.getP()); + } + if (!other.getSValue().isEmpty()) { + sValue_ = other.sValue_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Measresults parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Measresults) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int p_; + + /** + * uint32 p = 1; + */ + public int getP() { + return p_; + } + + /** + * uint32 p = 1; + */ + public Builder setP(int value) { + + p_ = value; + onChanged(); + return this; + } + + /** + * uint32 p = 1; + */ + public Builder clearP() { + + p_ = 0; + onChanged(); + return this; + } + + private java.lang.Object sValue_ = ""; + + /** + * string sValue = 2; + */ + public java.lang.String getSValue() { + java.lang.Object ref = sValue_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + sValue_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string sValue = 2; + */ + public com.google.protobuf.ByteString getSValueBytes() { + java.lang.Object ref = sValue_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + sValue_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string sValue = 2; + */ + public Builder setSValue(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + sValue_ = value; + onChanged(); + return this; + } + + /** + * string sValue = 2; + */ + public Builder clearSValue() { + + sValue_ = getDefaultInstance().getSValue(); + onChanged(); + return this; + } + + /** + * string sValue = 2; + */ + public Builder setSValueBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + sValue_ = value; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Measresults) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Measresults) + private static final PmProtoGenerated.PmRopFile.Measresults DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Measresults(); + } + + public static PmProtoGenerated.PmRopFile.Measresults getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Measresults parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Measresults(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measresults getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeasvalueslistOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Measvalueslist) + com.google.protobuf.MessageOrBuilder { + + /** + * string measObjInstId = 1; + */ + java.lang.String getMeasObjInstId(); + + /** + * string measObjInstId = 1; + */ + com.google.protobuf.ByteString getMeasObjInstIdBytes(); + + /** + * string suspectFlag = 2; + */ + java.lang.String getSuspectFlag(); + + /** + * string suspectFlag = 2; + */ + com.google.protobuf.ByteString getSuspectFlagBytes(); + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + java.util.List getMeasResultsList(); + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + PmProtoGenerated.PmRopFile.Measresults getMeasResults(int index); + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + int getMeasResultsCount(); + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + java.util.List getMeasResultsOrBuilderList(); + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + PmProtoGenerated.PmRopFile.MeasresultsOrBuilder getMeasResultsOrBuilder(int index); + } + + /** + * Protobuf type {@code PmRopFile.Measvalueslist} + */ + public static final class Measvalueslist extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Measvalueslist) + MeasvalueslistOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Measvalueslist.newBuilder() to construct. + private Measvalueslist(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Measvalueslist() { + measObjInstId_ = ""; + suspectFlag_ = ""; + measResults_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Measvalueslist(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + + measObjInstId_ = s; + break; + } + case 18: { + java.lang.String s = input.readStringRequireUtf8(); + + suspectFlag_ = s; + break; + } + case 26: { + if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + measResults_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000004; + } + measResults_.add(input.readMessage(PmProtoGenerated.PmRopFile.Measresults.parser(), + extensionRegistry)); + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + measResults_ = java.util.Collections.unmodifiableList(measResults_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measvalueslist_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measvalueslist_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measvalueslist.class, + PmProtoGenerated.PmRopFile.Measvalueslist.Builder.class); + } + + private int bitField0_; + public static final int MEASOBJINSTID_FIELD_NUMBER = 1; + private volatile java.lang.Object measObjInstId_; + + /** + * string measObjInstId = 1; + */ + public java.lang.String getMeasObjInstId() { + java.lang.Object ref = measObjInstId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measObjInstId_ = s; + return s; + } + } + + /** + * string measObjInstId = 1; + */ + public com.google.protobuf.ByteString getMeasObjInstIdBytes() { + java.lang.Object ref = measObjInstId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measObjInstId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SUSPECTFLAG_FIELD_NUMBER = 2; + private volatile java.lang.Object suspectFlag_; + + /** + * string suspectFlag = 2; + */ + public java.lang.String getSuspectFlag() { + java.lang.Object ref = suspectFlag_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + suspectFlag_ = s; + return s; + } + } + + /** + * string suspectFlag = 2; + */ + public com.google.protobuf.ByteString getSuspectFlagBytes() { + java.lang.Object ref = suspectFlag_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + suspectFlag_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MEASRESULTS_FIELD_NUMBER = 3; + private java.util.List measResults_; + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public java.util.List getMeasResultsList() { + return measResults_; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public java.util.List getMeasResultsOrBuilderList() { + return measResults_; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public int getMeasResultsCount() { + return measResults_.size(); + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.Measresults getMeasResults(int index) { + return measResults_.get(index); + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.MeasresultsOrBuilder getMeasResultsOrBuilder(int index) { + return measResults_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getMeasObjInstIdBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, measObjInstId_); + } + if (!getSuspectFlagBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, suspectFlag_); + } + for (int i = 0; i < measResults_.size(); i++) { + output.writeMessage(3, measResults_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (!getMeasObjInstIdBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, measObjInstId_); + } + if (!getSuspectFlagBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, suspectFlag_); + } + for (int i = 0; i < measResults_.size(); i++) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(3, measResults_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Measvalueslist)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Measvalueslist other = (PmProtoGenerated.PmRopFile.Measvalueslist) obj; + + boolean result = true; + result = result && getMeasObjInstId().equals(other.getMeasObjInstId()); + result = result && getSuspectFlag().equals(other.getSuspectFlag()); + result = result && getMeasResultsList().equals(other.getMeasResultsList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + MEASOBJINSTID_FIELD_NUMBER; + hash = (53 * hash) + getMeasObjInstId().hashCode(); + hash = (37 * hash) + SUSPECTFLAG_FIELD_NUMBER; + hash = (53 * hash) + getSuspectFlag().hashCode(); + if (getMeasResultsCount() > 0) { + hash = (37 * hash) + MEASRESULTS_FIELD_NUMBER; + hash = (53 * hash) + getMeasResultsList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Measvalueslist prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Measvalueslist} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Measvalueslist) + PmProtoGenerated.PmRopFile.MeasvalueslistOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measvalueslist_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measvalueslist_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measvalueslist.class, + PmProtoGenerated.PmRopFile.Measvalueslist.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Measvalueslist.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getMeasResultsFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + measObjInstId_ = ""; + + suspectFlag_ = ""; + + if (measResultsBuilder_ == null) { + measResults_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + } else { + measResultsBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Measvalueslist_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measvalueslist getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Measvalueslist.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measvalueslist build() { + PmProtoGenerated.PmRopFile.Measvalueslist result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measvalueslist buildPartial() { + PmProtoGenerated.PmRopFile.Measvalueslist result = + new PmProtoGenerated.PmRopFile.Measvalueslist(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + result.measObjInstId_ = measObjInstId_; + result.suspectFlag_ = suspectFlag_; + if (measResultsBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { + measResults_ = java.util.Collections.unmodifiableList(measResults_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.measResults_ = measResults_; + } else { + result.measResults_ = measResultsBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Measvalueslist) { + return mergeFrom((PmProtoGenerated.PmRopFile.Measvalueslist) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Measvalueslist other) { + if (other == PmProtoGenerated.PmRopFile.Measvalueslist.getDefaultInstance()) + return this; + if (!other.getMeasObjInstId().isEmpty()) { + measObjInstId_ = other.measObjInstId_; + onChanged(); + } + if (!other.getSuspectFlag().isEmpty()) { + suspectFlag_ = other.suspectFlag_; + onChanged(); + } + if (measResultsBuilder_ == null) { + if (!other.measResults_.isEmpty()) { + if (measResults_.isEmpty()) { + measResults_ = other.measResults_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureMeasResultsIsMutable(); + measResults_.addAll(other.measResults_); + } + onChanged(); + } + } else { + if (!other.measResults_.isEmpty()) { + if (measResultsBuilder_.isEmpty()) { + measResultsBuilder_.dispose(); + measResultsBuilder_ = null; + measResults_ = other.measResults_; + bitField0_ = (bitField0_ & ~0x00000004); + measResultsBuilder_ = com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders + ? getMeasResultsFieldBuilder() + : null; + } else { + measResultsBuilder_.addAllMessages(other.measResults_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Measvalueslist parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Measvalueslist) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object measObjInstId_ = ""; + + /** + * string measObjInstId = 1; + */ + public java.lang.String getMeasObjInstId() { + java.lang.Object ref = measObjInstId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measObjInstId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string measObjInstId = 1; + */ + public com.google.protobuf.ByteString getMeasObjInstIdBytes() { + java.lang.Object ref = measObjInstId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measObjInstId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string measObjInstId = 1; + */ + public Builder setMeasObjInstId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + measObjInstId_ = value; + onChanged(); + return this; + } + + /** + * string measObjInstId = 1; + */ + public Builder clearMeasObjInstId() { + + measObjInstId_ = getDefaultInstance().getMeasObjInstId(); + onChanged(); + return this; + } + + /** + * string measObjInstId = 1; + */ + public Builder setMeasObjInstIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + measObjInstId_ = value; + onChanged(); + return this; + } + + private java.lang.Object suspectFlag_ = ""; + + /** + * string suspectFlag = 2; + */ + public java.lang.String getSuspectFlag() { + java.lang.Object ref = suspectFlag_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + suspectFlag_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string suspectFlag = 2; + */ + public com.google.protobuf.ByteString getSuspectFlagBytes() { + java.lang.Object ref = suspectFlag_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + suspectFlag_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string suspectFlag = 2; + */ + public Builder setSuspectFlag(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + suspectFlag_ = value; + onChanged(); + return this; + } + + /** + * string suspectFlag = 2; + */ + public Builder clearSuspectFlag() { + + suspectFlag_ = getDefaultInstance().getSuspectFlag(); + onChanged(); + return this; + } + + /** + * string suspectFlag = 2; + */ + public Builder setSuspectFlagBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + suspectFlag_ = value; + onChanged(); + return this; + } + + private java.util.List measResults_ = + java.util.Collections.emptyList(); + + private void ensureMeasResultsIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + measResults_ = new java.util.ArrayList(measResults_); + bitField0_ |= 0x00000004; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3 measResultsBuilder_; + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public java.util.List getMeasResultsList() { + if (measResultsBuilder_ == null) { + return java.util.Collections.unmodifiableList(measResults_); + } else { + return measResultsBuilder_.getMessageList(); + } + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public int getMeasResultsCount() { + if (measResultsBuilder_ == null) { + return measResults_.size(); + } else { + return measResultsBuilder_.getCount(); + } + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.Measresults getMeasResults(int index) { + if (measResultsBuilder_ == null) { + return measResults_.get(index); + } else { + return measResultsBuilder_.getMessage(index); + } + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder setMeasResults(int index, PmProtoGenerated.PmRopFile.Measresults value) { + if (measResultsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasResultsIsMutable(); + measResults_.set(index, value); + onChanged(); + } else { + measResultsBuilder_.setMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder setMeasResults(int index, + PmProtoGenerated.PmRopFile.Measresults.Builder builderForValue) { + if (measResultsBuilder_ == null) { + ensureMeasResultsIsMutable(); + measResults_.set(index, builderForValue.build()); + onChanged(); + } else { + measResultsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder addMeasResults(PmProtoGenerated.PmRopFile.Measresults value) { + if (measResultsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasResultsIsMutable(); + measResults_.add(value); + onChanged(); + } else { + measResultsBuilder_.addMessage(value); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder addMeasResults(int index, PmProtoGenerated.PmRopFile.Measresults value) { + if (measResultsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasResultsIsMutable(); + measResults_.add(index, value); + onChanged(); + } else { + measResultsBuilder_.addMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder addMeasResults(PmProtoGenerated.PmRopFile.Measresults.Builder builderForValue) { + if (measResultsBuilder_ == null) { + ensureMeasResultsIsMutable(); + measResults_.add(builderForValue.build()); + onChanged(); + } else { + measResultsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder addMeasResults(int index, + PmProtoGenerated.PmRopFile.Measresults.Builder builderForValue) { + if (measResultsBuilder_ == null) { + ensureMeasResultsIsMutable(); + measResults_.add(index, builderForValue.build()); + onChanged(); + } else { + measResultsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder addAllMeasResults( + java.lang.Iterable values) { + if (measResultsBuilder_ == null) { + ensureMeasResultsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll(values, measResults_); + onChanged(); + } else { + measResultsBuilder_.addAllMessages(values); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder clearMeasResults() { + if (measResultsBuilder_ == null) { + measResults_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + } else { + measResultsBuilder_.clear(); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public Builder removeMeasResults(int index) { + if (measResultsBuilder_ == null) { + ensureMeasResultsIsMutable(); + measResults_.remove(index); + onChanged(); + } else { + measResultsBuilder_.remove(index); + } + return this; + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.Measresults.Builder getMeasResultsBuilder(int index) { + return getMeasResultsFieldBuilder().getBuilder(index); + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.MeasresultsOrBuilder getMeasResultsOrBuilder(int index) { + if (measResultsBuilder_ == null) { + return measResults_.get(index); + } else { + return measResultsBuilder_.getMessageOrBuilder(index); + } + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public java.util.List getMeasResultsOrBuilderList() { + if (measResultsBuilder_ != null) { + return measResultsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(measResults_); + } + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.Measresults.Builder addMeasResultsBuilder() { + return getMeasResultsFieldBuilder() + .addBuilder(PmProtoGenerated.PmRopFile.Measresults.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public PmProtoGenerated.PmRopFile.Measresults.Builder addMeasResultsBuilder(int index) { + return getMeasResultsFieldBuilder().addBuilder(index, + PmProtoGenerated.PmRopFile.Measresults.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measresults measResults = 3; + */ + public java.util.List getMeasResultsBuilderList() { + return getMeasResultsFieldBuilder().getBuilderList(); + } + + private com.google.protobuf.RepeatedFieldBuilderV3 getMeasResultsFieldBuilder() { + if (measResultsBuilder_ == null) { + measResultsBuilder_ = + new com.google.protobuf.RepeatedFieldBuilderV3( + measResults_, ((bitField0_ & 0x00000004) == 0x00000004), getParentForChildren(), + isClean()); + measResults_ = null; + } + return measResultsBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Measvalueslist) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Measvalueslist) + private static final PmProtoGenerated.PmRopFile.Measvalueslist DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Measvalueslist(); + } + + public static PmProtoGenerated.PmRopFile.Measvalueslist getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Measvalueslist parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Measvalueslist(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measvalueslist getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeasinfolistOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Measinfolist) + com.google.protobuf.MessageOrBuilder { + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + boolean hasMeasInfoId(); + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + PmProtoGenerated.PmRopFile.Measinfoid getMeasInfoId(); + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + PmProtoGenerated.PmRopFile.MeasinfoidOrBuilder getMeasInfoIdOrBuilder(); + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + boolean hasMeasTypes(); + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + PmProtoGenerated.PmRopFile.Meastypes getMeasTypes(); + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + PmProtoGenerated.PmRopFile.MeastypesOrBuilder getMeasTypesOrBuilder(); + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + java.util.List getMeasValuesListList(); + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + PmProtoGenerated.PmRopFile.Measvalueslist getMeasValuesList(int index); + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + int getMeasValuesListCount(); + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + java.util.List getMeasValuesListOrBuilderList(); + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + PmProtoGenerated.PmRopFile.MeasvalueslistOrBuilder getMeasValuesListOrBuilder(int index); + } + + /** + * Protobuf type {@code PmRopFile.Measinfolist} + */ + public static final class Measinfolist extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Measinfolist) + MeasinfolistOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Measinfolist.newBuilder() to construct. + private Measinfolist(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Measinfolist() { + measValuesList_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Measinfolist(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + PmProtoGenerated.PmRopFile.Measinfoid.Builder subBuilder = null; + if (measInfoId_ != null) { + subBuilder = measInfoId_.toBuilder(); + } + measInfoId_ = input.readMessage(PmProtoGenerated.PmRopFile.Measinfoid.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(measInfoId_); + measInfoId_ = subBuilder.buildPartial(); + } + + break; + } + case 18: { + PmProtoGenerated.PmRopFile.Meastypes.Builder subBuilder = null; + if (measTypes_ != null) { + subBuilder = measTypes_.toBuilder(); + } + measTypes_ = input.readMessage(PmProtoGenerated.PmRopFile.Meastypes.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(measTypes_); + measTypes_ = subBuilder.buildPartial(); + } + + break; + } + case 26: { + if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + measValuesList_ = + new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000004; + } + measValuesList_.add(input.readMessage( + PmProtoGenerated.PmRopFile.Measvalueslist.parser(), extensionRegistry)); + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + measValuesList_ = java.util.Collections.unmodifiableList(measValuesList_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfolist_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfolist_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measinfolist.class, + PmProtoGenerated.PmRopFile.Measinfolist.Builder.class); + } + + private int bitField0_; + public static final int MEASINFOID_FIELD_NUMBER = 1; + private PmProtoGenerated.PmRopFile.Measinfoid measInfoId_; + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public boolean hasMeasInfoId() { + return measInfoId_ != null; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public PmProtoGenerated.PmRopFile.Measinfoid getMeasInfoId() { + return measInfoId_ == null ? PmProtoGenerated.PmRopFile.Measinfoid.getDefaultInstance() : measInfoId_; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public PmProtoGenerated.PmRopFile.MeasinfoidOrBuilder getMeasInfoIdOrBuilder() { + return getMeasInfoId(); + } + + public static final int MEASTYPES_FIELD_NUMBER = 2; + private PmProtoGenerated.PmRopFile.Meastypes measTypes_; + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public boolean hasMeasTypes() { + return measTypes_ != null; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public PmProtoGenerated.PmRopFile.Meastypes getMeasTypes() { + return measTypes_ == null ? PmProtoGenerated.PmRopFile.Meastypes.getDefaultInstance() : measTypes_; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public PmProtoGenerated.PmRopFile.MeastypesOrBuilder getMeasTypesOrBuilder() { + return getMeasTypes(); + } + + public static final int MEASVALUESLIST_FIELD_NUMBER = 3; + private java.util.List measValuesList_; + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public java.util.List getMeasValuesListList() { + return measValuesList_; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public java.util.List getMeasValuesListOrBuilderList() { + return measValuesList_; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public int getMeasValuesListCount() { + return measValuesList_.size(); + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.Measvalueslist getMeasValuesList(int index) { + return measValuesList_.get(index); + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.MeasvalueslistOrBuilder getMeasValuesListOrBuilder(int index) { + return measValuesList_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (measInfoId_ != null) { + output.writeMessage(1, getMeasInfoId()); + } + if (measTypes_ != null) { + output.writeMessage(2, getMeasTypes()); + } + for (int i = 0; i < measValuesList_.size(); i++) { + output.writeMessage(3, measValuesList_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (measInfoId_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(1, getMeasInfoId()); + } + if (measTypes_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(2, getMeasTypes()); + } + for (int i = 0; i < measValuesList_.size(); i++) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(3, measValuesList_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Measinfolist)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Measinfolist other = (PmProtoGenerated.PmRopFile.Measinfolist) obj; + + boolean result = true; + result = result && (hasMeasInfoId() == other.hasMeasInfoId()); + if (hasMeasInfoId()) { + result = result && getMeasInfoId().equals(other.getMeasInfoId()); + } + result = result && (hasMeasTypes() == other.hasMeasTypes()); + if (hasMeasTypes()) { + result = result && getMeasTypes().equals(other.getMeasTypes()); + } + result = result && getMeasValuesListList().equals(other.getMeasValuesListList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasMeasInfoId()) { + hash = (37 * hash) + MEASINFOID_FIELD_NUMBER; + hash = (53 * hash) + getMeasInfoId().hashCode(); + } + if (hasMeasTypes()) { + hash = (37 * hash) + MEASTYPES_FIELD_NUMBER; + hash = (53 * hash) + getMeasTypes().hashCode(); + } + if (getMeasValuesListCount() > 0) { + hash = (37 * hash) + MEASVALUESLIST_FIELD_NUMBER; + hash = (53 * hash) + getMeasValuesListList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Measinfolist prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Measinfolist} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Measinfolist) + PmProtoGenerated.PmRopFile.MeasinfolistOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfolist_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfolist_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measinfolist.class, + PmProtoGenerated.PmRopFile.Measinfolist.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Measinfolist.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getMeasValuesListFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (measInfoIdBuilder_ == null) { + measInfoId_ = null; + } else { + measInfoId_ = null; + measInfoIdBuilder_ = null; + } + if (measTypesBuilder_ == null) { + measTypes_ = null; + } else { + measTypes_ = null; + measTypesBuilder_ = null; + } + if (measValuesListBuilder_ == null) { + measValuesList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + } else { + measValuesListBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Measinfolist_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfolist getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Measinfolist.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfolist build() { + PmProtoGenerated.PmRopFile.Measinfolist result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfolist buildPartial() { + PmProtoGenerated.PmRopFile.Measinfolist result = new PmProtoGenerated.PmRopFile.Measinfolist(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (measInfoIdBuilder_ == null) { + result.measInfoId_ = measInfoId_; + } else { + result.measInfoId_ = measInfoIdBuilder_.build(); + } + if (measTypesBuilder_ == null) { + result.measTypes_ = measTypes_; + } else { + result.measTypes_ = measTypesBuilder_.build(); + } + if (measValuesListBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { + measValuesList_ = java.util.Collections.unmodifiableList(measValuesList_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.measValuesList_ = measValuesList_; + } else { + result.measValuesList_ = measValuesListBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Measinfolist) { + return mergeFrom((PmProtoGenerated.PmRopFile.Measinfolist) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Measinfolist other) { + if (other == PmProtoGenerated.PmRopFile.Measinfolist.getDefaultInstance()) + return this; + if (other.hasMeasInfoId()) { + mergeMeasInfoId(other.getMeasInfoId()); + } + if (other.hasMeasTypes()) { + mergeMeasTypes(other.getMeasTypes()); + } + if (measValuesListBuilder_ == null) { + if (!other.measValuesList_.isEmpty()) { + if (measValuesList_.isEmpty()) { + measValuesList_ = other.measValuesList_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureMeasValuesListIsMutable(); + measValuesList_.addAll(other.measValuesList_); + } + onChanged(); + } + } else { + if (!other.measValuesList_.isEmpty()) { + if (measValuesListBuilder_.isEmpty()) { + measValuesListBuilder_.dispose(); + measValuesListBuilder_ = null; + measValuesList_ = other.measValuesList_; + bitField0_ = (bitField0_ & ~0x00000004); + measValuesListBuilder_ = com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders + ? getMeasValuesListFieldBuilder() + : null; + } else { + measValuesListBuilder_.addAllMessages(other.measValuesList_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Measinfolist parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Measinfolist) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private PmProtoGenerated.PmRopFile.Measinfoid measInfoId_ = null; + private com.google.protobuf.SingleFieldBuilderV3 measInfoIdBuilder_; + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public boolean hasMeasInfoId() { + return measInfoIdBuilder_ != null || measInfoId_ != null; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public PmProtoGenerated.PmRopFile.Measinfoid getMeasInfoId() { + if (measInfoIdBuilder_ == null) { + return measInfoId_ == null ? PmProtoGenerated.PmRopFile.Measinfoid.getDefaultInstance() + : measInfoId_; + } else { + return measInfoIdBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public Builder setMeasInfoId(PmProtoGenerated.PmRopFile.Measinfoid value) { + if (measInfoIdBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + measInfoId_ = value; + onChanged(); + } else { + measInfoIdBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public Builder setMeasInfoId(PmProtoGenerated.PmRopFile.Measinfoid.Builder builderForValue) { + if (measInfoIdBuilder_ == null) { + measInfoId_ = builderForValue.build(); + onChanged(); + } else { + measInfoIdBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public Builder mergeMeasInfoId(PmProtoGenerated.PmRopFile.Measinfoid value) { + if (measInfoIdBuilder_ == null) { + if (measInfoId_ != null) { + measInfoId_ = PmProtoGenerated.PmRopFile.Measinfoid.newBuilder(measInfoId_).mergeFrom(value) + .buildPartial(); + } else { + measInfoId_ = value; + } + onChanged(); + } else { + measInfoIdBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public Builder clearMeasInfoId() { + if (measInfoIdBuilder_ == null) { + measInfoId_ = null; + onChanged(); + } else { + measInfoId_ = null; + measInfoIdBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public PmProtoGenerated.PmRopFile.Measinfoid.Builder getMeasInfoIdBuilder() { + + onChanged(); + return getMeasInfoIdFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + public PmProtoGenerated.PmRopFile.MeasinfoidOrBuilder getMeasInfoIdOrBuilder() { + if (measInfoIdBuilder_ != null) { + return measInfoIdBuilder_.getMessageOrBuilder(); + } else { + return measInfoId_ == null ? PmProtoGenerated.PmRopFile.Measinfoid.getDefaultInstance() + : measInfoId_; + } + } + + /** + * .PmRopFile.Measinfoid measInfoId = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3 getMeasInfoIdFieldBuilder() { + if (measInfoIdBuilder_ == null) { + measInfoIdBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getMeasInfoId(), getParentForChildren(), isClean()); + measInfoId_ = null; + } + return measInfoIdBuilder_; + } + + private PmProtoGenerated.PmRopFile.Meastypes measTypes_ = null; + private com.google.protobuf.SingleFieldBuilderV3 measTypesBuilder_; + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public boolean hasMeasTypes() { + return measTypesBuilder_ != null || measTypes_ != null; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public PmProtoGenerated.PmRopFile.Meastypes getMeasTypes() { + if (measTypesBuilder_ == null) { + return measTypes_ == null ? PmProtoGenerated.PmRopFile.Meastypes.getDefaultInstance() + : measTypes_; + } else { + return measTypesBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public Builder setMeasTypes(PmProtoGenerated.PmRopFile.Meastypes value) { + if (measTypesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + measTypes_ = value; + onChanged(); + } else { + measTypesBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public Builder setMeasTypes(PmProtoGenerated.PmRopFile.Meastypes.Builder builderForValue) { + if (measTypesBuilder_ == null) { + measTypes_ = builderForValue.build(); + onChanged(); + } else { + measTypesBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public Builder mergeMeasTypes(PmProtoGenerated.PmRopFile.Meastypes value) { + if (measTypesBuilder_ == null) { + if (measTypes_ != null) { + measTypes_ = PmProtoGenerated.PmRopFile.Meastypes.newBuilder(measTypes_).mergeFrom(value) + .buildPartial(); + } else { + measTypes_ = value; + } + onChanged(); + } else { + measTypesBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public Builder clearMeasTypes() { + if (measTypesBuilder_ == null) { + measTypes_ = null; + onChanged(); + } else { + measTypes_ = null; + measTypesBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public PmProtoGenerated.PmRopFile.Meastypes.Builder getMeasTypesBuilder() { + + onChanged(); + return getMeasTypesFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + public PmProtoGenerated.PmRopFile.MeastypesOrBuilder getMeasTypesOrBuilder() { + if (measTypesBuilder_ != null) { + return measTypesBuilder_.getMessageOrBuilder(); + } else { + return measTypes_ == null ? PmProtoGenerated.PmRopFile.Meastypes.getDefaultInstance() + : measTypes_; + } + } + + /** + * .PmRopFile.Meastypes measTypes = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3 getMeasTypesFieldBuilder() { + if (measTypesBuilder_ == null) { + measTypesBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getMeasTypes(), getParentForChildren(), isClean()); + measTypes_ = null; + } + return measTypesBuilder_; + } + + private java.util.List measValuesList_ = + java.util.Collections.emptyList(); + + private void ensureMeasValuesListIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + measValuesList_ = + new java.util.ArrayList(measValuesList_); + bitField0_ |= 0x00000004; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3 measValuesListBuilder_; + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public java.util.List getMeasValuesListList() { + if (measValuesListBuilder_ == null) { + return java.util.Collections.unmodifiableList(measValuesList_); + } else { + return measValuesListBuilder_.getMessageList(); + } + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public int getMeasValuesListCount() { + if (measValuesListBuilder_ == null) { + return measValuesList_.size(); + } else { + return measValuesListBuilder_.getCount(); + } + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.Measvalueslist getMeasValuesList(int index) { + if (measValuesListBuilder_ == null) { + return measValuesList_.get(index); + } else { + return measValuesListBuilder_.getMessage(index); + } + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder setMeasValuesList(int index, PmProtoGenerated.PmRopFile.Measvalueslist value) { + if (measValuesListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasValuesListIsMutable(); + measValuesList_.set(index, value); + onChanged(); + } else { + measValuesListBuilder_.setMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder setMeasValuesList(int index, + PmProtoGenerated.PmRopFile.Measvalueslist.Builder builderForValue) { + if (measValuesListBuilder_ == null) { + ensureMeasValuesListIsMutable(); + measValuesList_.set(index, builderForValue.build()); + onChanged(); + } else { + measValuesListBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder addMeasValuesList(PmProtoGenerated.PmRopFile.Measvalueslist value) { + if (measValuesListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasValuesListIsMutable(); + measValuesList_.add(value); + onChanged(); + } else { + measValuesListBuilder_.addMessage(value); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder addMeasValuesList(int index, PmProtoGenerated.PmRopFile.Measvalueslist value) { + if (measValuesListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasValuesListIsMutable(); + measValuesList_.add(index, value); + onChanged(); + } else { + measValuesListBuilder_.addMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder addMeasValuesList(PmProtoGenerated.PmRopFile.Measvalueslist.Builder builderForValue) { + if (measValuesListBuilder_ == null) { + ensureMeasValuesListIsMutable(); + measValuesList_.add(builderForValue.build()); + onChanged(); + } else { + measValuesListBuilder_.addMessage(builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder addMeasValuesList(int index, + PmProtoGenerated.PmRopFile.Measvalueslist.Builder builderForValue) { + if (measValuesListBuilder_ == null) { + ensureMeasValuesListIsMutable(); + measValuesList_.add(index, builderForValue.build()); + onChanged(); + } else { + measValuesListBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder addAllMeasValuesList( + java.lang.Iterable values) { + if (measValuesListBuilder_ == null) { + ensureMeasValuesListIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll(values, measValuesList_); + onChanged(); + } else { + measValuesListBuilder_.addAllMessages(values); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder clearMeasValuesList() { + if (measValuesListBuilder_ == null) { + measValuesList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + } else { + measValuesListBuilder_.clear(); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public Builder removeMeasValuesList(int index) { + if (measValuesListBuilder_ == null) { + ensureMeasValuesListIsMutable(); + measValuesList_.remove(index); + onChanged(); + } else { + measValuesListBuilder_.remove(index); + } + return this; + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.Measvalueslist.Builder getMeasValuesListBuilder(int index) { + return getMeasValuesListFieldBuilder().getBuilder(index); + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.MeasvalueslistOrBuilder getMeasValuesListOrBuilder(int index) { + if (measValuesListBuilder_ == null) { + return measValuesList_.get(index); + } else { + return measValuesListBuilder_.getMessageOrBuilder(index); + } + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public java.util.List getMeasValuesListOrBuilderList() { + if (measValuesListBuilder_ != null) { + return measValuesListBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(measValuesList_); + } + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.Measvalueslist.Builder addMeasValuesListBuilder() { + return getMeasValuesListFieldBuilder() + .addBuilder(PmProtoGenerated.PmRopFile.Measvalueslist.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public PmProtoGenerated.PmRopFile.Measvalueslist.Builder addMeasValuesListBuilder(int index) { + return getMeasValuesListFieldBuilder().addBuilder(index, + PmProtoGenerated.PmRopFile.Measvalueslist.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measvalueslist measValuesList = 3; + */ + public java.util.List getMeasValuesListBuilderList() { + return getMeasValuesListFieldBuilder().getBuilderList(); + } + + private com.google.protobuf.RepeatedFieldBuilderV3 getMeasValuesListFieldBuilder() { + if (measValuesListBuilder_ == null) { + measValuesListBuilder_ = + new com.google.protobuf.RepeatedFieldBuilderV3( + measValuesList_, ((bitField0_ & 0x00000004) == 0x00000004), + getParentForChildren(), isClean()); + measValuesList_ = null; + } + return measValuesListBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Measinfolist) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Measinfolist) + private static final PmProtoGenerated.PmRopFile.Measinfolist DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Measinfolist(); + } + + public static PmProtoGenerated.PmRopFile.Measinfolist getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Measinfolist parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Measinfolist(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measinfolist getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface MeasdatacollectionOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Measdatacollection) + com.google.protobuf.MessageOrBuilder { + + /** + * uint32 granularityPeriod = 1; + */ + int getGranularityPeriod(); + + /** + * string measuredEntityUserName = 2; + */ + java.lang.String getMeasuredEntityUserName(); + + /** + * string measuredEntityUserName = 2; + */ + com.google.protobuf.ByteString getMeasuredEntityUserNameBytes(); + + /** + * string measuredEntityDn = 3; + */ + java.lang.String getMeasuredEntityDn(); + + /** + * string measuredEntityDn = 3; + */ + com.google.protobuf.ByteString getMeasuredEntityDnBytes(); + + /** + * string measuredEntitySoftwareVersion = 4; + */ + java.lang.String getMeasuredEntitySoftwareVersion(); + + /** + * string measuredEntitySoftwareVersion = 4; + */ + com.google.protobuf.ByteString getMeasuredEntitySoftwareVersionBytes(); + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + java.util.List getMeasInfoListList(); + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + PmProtoGenerated.PmRopFile.Measinfolist getMeasInfoList(int index); + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + int getMeasInfoListCount(); + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + java.util.List getMeasInfoListOrBuilderList(); + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + PmProtoGenerated.PmRopFile.MeasinfolistOrBuilder getMeasInfoListOrBuilder(int index); + } + + /** + * Protobuf type {@code PmRopFile.Measdatacollection} + */ + public static final class Measdatacollection extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Measdatacollection) + MeasdatacollectionOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Measdatacollection.newBuilder() to construct. + private Measdatacollection(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Measdatacollection() { + granularityPeriod_ = 0; + measuredEntityUserName_ = ""; + measuredEntityDn_ = ""; + measuredEntitySoftwareVersion_ = ""; + measInfoList_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Measdatacollection(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + + granularityPeriod_ = input.readUInt32(); + break; + } + case 18: { + java.lang.String s = input.readStringRequireUtf8(); + + measuredEntityUserName_ = s; + break; + } + case 26: { + java.lang.String s = input.readStringRequireUtf8(); + + measuredEntityDn_ = s; + break; + } + case 34: { + java.lang.String s = input.readStringRequireUtf8(); + + measuredEntitySoftwareVersion_ = s; + break; + } + case 42: { + if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + measInfoList_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000010; + } + measInfoList_.add(input.readMessage(PmProtoGenerated.PmRopFile.Measinfolist.parser(), + extensionRegistry)); + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + measInfoList_ = java.util.Collections.unmodifiableList(measInfoList_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measdatacollection_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measdatacollection_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measdatacollection.class, + PmProtoGenerated.PmRopFile.Measdatacollection.Builder.class); + } + + private int bitField0_; + public static final int GRANULARITYPERIOD_FIELD_NUMBER = 1; + private int granularityPeriod_; + + /** + * uint32 granularityPeriod = 1; + */ + public int getGranularityPeriod() { + return granularityPeriod_; + } + + public static final int MEASUREDENTITYUSERNAME_FIELD_NUMBER = 2; + private volatile java.lang.Object measuredEntityUserName_; + + /** + * string measuredEntityUserName = 2; + */ + public java.lang.String getMeasuredEntityUserName() { + java.lang.Object ref = measuredEntityUserName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntityUserName_ = s; + return s; + } + } + + /** + * string measuredEntityUserName = 2; + */ + public com.google.protobuf.ByteString getMeasuredEntityUserNameBytes() { + java.lang.Object ref = measuredEntityUserName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntityUserName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MEASUREDENTITYDN_FIELD_NUMBER = 3; + private volatile java.lang.Object measuredEntityDn_; + + /** + * string measuredEntityDn = 3; + */ + public java.lang.String getMeasuredEntityDn() { + java.lang.Object ref = measuredEntityDn_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntityDn_ = s; + return s; + } + } + + /** + * string measuredEntityDn = 3; + */ + public com.google.protobuf.ByteString getMeasuredEntityDnBytes() { + java.lang.Object ref = measuredEntityDn_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntityDn_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MEASUREDENTITYSOFTWAREVERSION_FIELD_NUMBER = 4; + private volatile java.lang.Object measuredEntitySoftwareVersion_; + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public java.lang.String getMeasuredEntitySoftwareVersion() { + java.lang.Object ref = measuredEntitySoftwareVersion_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntitySoftwareVersion_ = s; + return s; + } + } + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public com.google.protobuf.ByteString getMeasuredEntitySoftwareVersionBytes() { + java.lang.Object ref = measuredEntitySoftwareVersion_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntitySoftwareVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MEASINFOLIST_FIELD_NUMBER = 5; + private java.util.List measInfoList_; + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public java.util.List getMeasInfoListList() { + return measInfoList_; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public java.util.List getMeasInfoListOrBuilderList() { + return measInfoList_; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public int getMeasInfoListCount() { + return measInfoList_.size(); + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.Measinfolist getMeasInfoList(int index) { + return measInfoList_.get(index); + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.MeasinfolistOrBuilder getMeasInfoListOrBuilder(int index) { + return measInfoList_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (granularityPeriod_ != 0) { + output.writeUInt32(1, granularityPeriod_); + } + if (!getMeasuredEntityUserNameBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, measuredEntityUserName_); + } + if (!getMeasuredEntityDnBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, measuredEntityDn_); + } + if (!getMeasuredEntitySoftwareVersionBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 4, measuredEntitySoftwareVersion_); + } + for (int i = 0; i < measInfoList_.size(); i++) { + output.writeMessage(5, measInfoList_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (granularityPeriod_ != 0) { + size += com.google.protobuf.CodedOutputStream.computeUInt32Size(1, granularityPeriod_); + } + if (!getMeasuredEntityUserNameBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, measuredEntityUserName_); + } + if (!getMeasuredEntityDnBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, measuredEntityDn_); + } + if (!getMeasuredEntitySoftwareVersionBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, measuredEntitySoftwareVersion_); + } + for (int i = 0; i < measInfoList_.size(); i++) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(5, measInfoList_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Measdatacollection)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Measdatacollection other = + (PmProtoGenerated.PmRopFile.Measdatacollection) obj; + + boolean result = true; + result = result && (getGranularityPeriod() == other.getGranularityPeriod()); + result = result && getMeasuredEntityUserName().equals(other.getMeasuredEntityUserName()); + result = result && getMeasuredEntityDn().equals(other.getMeasuredEntityDn()); + result = result && getMeasuredEntitySoftwareVersion().equals(other.getMeasuredEntitySoftwareVersion()); + result = result && getMeasInfoListList().equals(other.getMeasInfoListList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + GRANULARITYPERIOD_FIELD_NUMBER; + hash = (53 * hash) + getGranularityPeriod(); + hash = (37 * hash) + MEASUREDENTITYUSERNAME_FIELD_NUMBER; + hash = (53 * hash) + getMeasuredEntityUserName().hashCode(); + hash = (37 * hash) + MEASUREDENTITYDN_FIELD_NUMBER; + hash = (53 * hash) + getMeasuredEntityDn().hashCode(); + hash = (37 * hash) + MEASUREDENTITYSOFTWAREVERSION_FIELD_NUMBER; + hash = (53 * hash) + getMeasuredEntitySoftwareVersion().hashCode(); + if (getMeasInfoListCount() > 0) { + hash = (37 * hash) + MEASINFOLIST_FIELD_NUMBER; + hash = (53 * hash) + getMeasInfoListList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Measdatacollection prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Measdatacollection} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Measdatacollection) + PmProtoGenerated.PmRopFile.MeasdatacollectionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Measdatacollection_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Measdatacollection_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Measdatacollection.class, + PmProtoGenerated.PmRopFile.Measdatacollection.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Measdatacollection.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getMeasInfoListFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + granularityPeriod_ = 0; + + measuredEntityUserName_ = ""; + + measuredEntityDn_ = ""; + + measuredEntitySoftwareVersion_ = ""; + + if (measInfoListBuilder_ == null) { + measInfoList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + } else { + measInfoListBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Measdatacollection_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measdatacollection getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Measdatacollection.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measdatacollection build() { + PmProtoGenerated.PmRopFile.Measdatacollection result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measdatacollection buildPartial() { + PmProtoGenerated.PmRopFile.Measdatacollection result = + new PmProtoGenerated.PmRopFile.Measdatacollection(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + result.granularityPeriod_ = granularityPeriod_; + result.measuredEntityUserName_ = measuredEntityUserName_; + result.measuredEntityDn_ = measuredEntityDn_; + result.measuredEntitySoftwareVersion_ = measuredEntitySoftwareVersion_; + if (measInfoListBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010)) { + measInfoList_ = java.util.Collections.unmodifiableList(measInfoList_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.measInfoList_ = measInfoList_; + } else { + result.measInfoList_ = measInfoListBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Measdatacollection) { + return mergeFrom((PmProtoGenerated.PmRopFile.Measdatacollection) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Measdatacollection other) { + if (other == PmProtoGenerated.PmRopFile.Measdatacollection.getDefaultInstance()) + return this; + if (other.getGranularityPeriod() != 0) { + setGranularityPeriod(other.getGranularityPeriod()); + } + if (!other.getMeasuredEntityUserName().isEmpty()) { + measuredEntityUserName_ = other.measuredEntityUserName_; + onChanged(); + } + if (!other.getMeasuredEntityDn().isEmpty()) { + measuredEntityDn_ = other.measuredEntityDn_; + onChanged(); + } + if (!other.getMeasuredEntitySoftwareVersion().isEmpty()) { + measuredEntitySoftwareVersion_ = other.measuredEntitySoftwareVersion_; + onChanged(); + } + if (measInfoListBuilder_ == null) { + if (!other.measInfoList_.isEmpty()) { + if (measInfoList_.isEmpty()) { + measInfoList_ = other.measInfoList_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureMeasInfoListIsMutable(); + measInfoList_.addAll(other.measInfoList_); + } + onChanged(); + } + } else { + if (!other.measInfoList_.isEmpty()) { + if (measInfoListBuilder_.isEmpty()) { + measInfoListBuilder_.dispose(); + measInfoListBuilder_ = null; + measInfoList_ = other.measInfoList_; + bitField0_ = (bitField0_ & ~0x00000010); + measInfoListBuilder_ = com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders + ? getMeasInfoListFieldBuilder() + : null; + } else { + measInfoListBuilder_.addAllMessages(other.measInfoList_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Measdatacollection parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Measdatacollection) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private int granularityPeriod_; + + /** + * uint32 granularityPeriod = 1; + */ + public int getGranularityPeriod() { + return granularityPeriod_; + } + + /** + * uint32 granularityPeriod = 1; + */ + public Builder setGranularityPeriod(int value) { + + granularityPeriod_ = value; + onChanged(); + return this; + } + + /** + * uint32 granularityPeriod = 1; + */ + public Builder clearGranularityPeriod() { + + granularityPeriod_ = 0; + onChanged(); + return this; + } + + private java.lang.Object measuredEntityUserName_ = ""; + + /** + * string measuredEntityUserName = 2; + */ + public java.lang.String getMeasuredEntityUserName() { + java.lang.Object ref = measuredEntityUserName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntityUserName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string measuredEntityUserName = 2; + */ + public com.google.protobuf.ByteString getMeasuredEntityUserNameBytes() { + java.lang.Object ref = measuredEntityUserName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntityUserName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string measuredEntityUserName = 2; + */ + public Builder setMeasuredEntityUserName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + measuredEntityUserName_ = value; + onChanged(); + return this; + } + + /** + * string measuredEntityUserName = 2; + */ + public Builder clearMeasuredEntityUserName() { + + measuredEntityUserName_ = getDefaultInstance().getMeasuredEntityUserName(); + onChanged(); + return this; + } + + /** + * string measuredEntityUserName = 2; + */ + public Builder setMeasuredEntityUserNameBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + measuredEntityUserName_ = value; + onChanged(); + return this; + } + + private java.lang.Object measuredEntityDn_ = ""; + + /** + * string measuredEntityDn = 3; + */ + public java.lang.String getMeasuredEntityDn() { + java.lang.Object ref = measuredEntityDn_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntityDn_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string measuredEntityDn = 3; + */ + public com.google.protobuf.ByteString getMeasuredEntityDnBytes() { + java.lang.Object ref = measuredEntityDn_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntityDn_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string measuredEntityDn = 3; + */ + public Builder setMeasuredEntityDn(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + measuredEntityDn_ = value; + onChanged(); + return this; + } + + /** + * string measuredEntityDn = 3; + */ + public Builder clearMeasuredEntityDn() { + + measuredEntityDn_ = getDefaultInstance().getMeasuredEntityDn(); + onChanged(); + return this; + } + + /** + * string measuredEntityDn = 3; + */ + public Builder setMeasuredEntityDnBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + measuredEntityDn_ = value; + onChanged(); + return this; + } + + private java.lang.Object measuredEntitySoftwareVersion_ = ""; + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public java.lang.String getMeasuredEntitySoftwareVersion() { + java.lang.Object ref = measuredEntitySoftwareVersion_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + measuredEntitySoftwareVersion_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public com.google.protobuf.ByteString getMeasuredEntitySoftwareVersionBytes() { + java.lang.Object ref = measuredEntitySoftwareVersion_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + measuredEntitySoftwareVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public Builder setMeasuredEntitySoftwareVersion(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + measuredEntitySoftwareVersion_ = value; + onChanged(); + return this; + } + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public Builder clearMeasuredEntitySoftwareVersion() { + + measuredEntitySoftwareVersion_ = getDefaultInstance().getMeasuredEntitySoftwareVersion(); + onChanged(); + return this; + } + + /** + * string measuredEntitySoftwareVersion = 4; + */ + public Builder setMeasuredEntitySoftwareVersionBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + measuredEntitySoftwareVersion_ = value; + onChanged(); + return this; + } + + private java.util.List measInfoList_ = + java.util.Collections.emptyList(); + + private void ensureMeasInfoListIsMutable() { + if (!((bitField0_ & 0x00000010) == 0x00000010)) { + measInfoList_ = new java.util.ArrayList(measInfoList_); + bitField0_ |= 0x00000010; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3 measInfoListBuilder_; + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public java.util.List getMeasInfoListList() { + if (measInfoListBuilder_ == null) { + return java.util.Collections.unmodifiableList(measInfoList_); + } else { + return measInfoListBuilder_.getMessageList(); + } + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public int getMeasInfoListCount() { + if (measInfoListBuilder_ == null) { + return measInfoList_.size(); + } else { + return measInfoListBuilder_.getCount(); + } + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.Measinfolist getMeasInfoList(int index) { + if (measInfoListBuilder_ == null) { + return measInfoList_.get(index); + } else { + return measInfoListBuilder_.getMessage(index); + } + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder setMeasInfoList(int index, PmProtoGenerated.PmRopFile.Measinfolist value) { + if (measInfoListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasInfoListIsMutable(); + measInfoList_.set(index, value); + onChanged(); + } else { + measInfoListBuilder_.setMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder setMeasInfoList(int index, + PmProtoGenerated.PmRopFile.Measinfolist.Builder builderForValue) { + if (measInfoListBuilder_ == null) { + ensureMeasInfoListIsMutable(); + measInfoList_.set(index, builderForValue.build()); + onChanged(); + } else { + measInfoListBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder addMeasInfoList(PmProtoGenerated.PmRopFile.Measinfolist value) { + if (measInfoListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasInfoListIsMutable(); + measInfoList_.add(value); + onChanged(); + } else { + measInfoListBuilder_.addMessage(value); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder addMeasInfoList(int index, PmProtoGenerated.PmRopFile.Measinfolist value) { + if (measInfoListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMeasInfoListIsMutable(); + measInfoList_.add(index, value); + onChanged(); + } else { + measInfoListBuilder_.addMessage(index, value); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder addMeasInfoList(PmProtoGenerated.PmRopFile.Measinfolist.Builder builderForValue) { + if (measInfoListBuilder_ == null) { + ensureMeasInfoListIsMutable(); + measInfoList_.add(builderForValue.build()); + onChanged(); + } else { + measInfoListBuilder_.addMessage(builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder addMeasInfoList(int index, + PmProtoGenerated.PmRopFile.Measinfolist.Builder builderForValue) { + if (measInfoListBuilder_ == null) { + ensureMeasInfoListIsMutable(); + measInfoList_.add(index, builderForValue.build()); + onChanged(); + } else { + measInfoListBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder addAllMeasInfoList( + java.lang.Iterable values) { + if (measInfoListBuilder_ == null) { + ensureMeasInfoListIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll(values, measInfoList_); + onChanged(); + } else { + measInfoListBuilder_.addAllMessages(values); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder clearMeasInfoList() { + if (measInfoListBuilder_ == null) { + measInfoList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + } else { + measInfoListBuilder_.clear(); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public Builder removeMeasInfoList(int index) { + if (measInfoListBuilder_ == null) { + ensureMeasInfoListIsMutable(); + measInfoList_.remove(index); + onChanged(); + } else { + measInfoListBuilder_.remove(index); + } + return this; + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.Measinfolist.Builder getMeasInfoListBuilder(int index) { + return getMeasInfoListFieldBuilder().getBuilder(index); + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.MeasinfolistOrBuilder getMeasInfoListOrBuilder(int index) { + if (measInfoListBuilder_ == null) { + return measInfoList_.get(index); + } else { + return measInfoListBuilder_.getMessageOrBuilder(index); + } + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public java.util.List getMeasInfoListOrBuilderList() { + if (measInfoListBuilder_ != null) { + return measInfoListBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(measInfoList_); + } + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.Measinfolist.Builder addMeasInfoListBuilder() { + return getMeasInfoListFieldBuilder() + .addBuilder(PmProtoGenerated.PmRopFile.Measinfolist.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public PmProtoGenerated.PmRopFile.Measinfolist.Builder addMeasInfoListBuilder(int index) { + return getMeasInfoListFieldBuilder().addBuilder(index, + PmProtoGenerated.PmRopFile.Measinfolist.getDefaultInstance()); + } + + /** + * repeated .PmRopFile.Measinfolist measInfoList = 5; + */ + public java.util.List getMeasInfoListBuilderList() { + return getMeasInfoListFieldBuilder().getBuilderList(); + } + + private com.google.protobuf.RepeatedFieldBuilderV3 getMeasInfoListFieldBuilder() { + if (measInfoListBuilder_ == null) { + measInfoListBuilder_ = + new com.google.protobuf.RepeatedFieldBuilderV3( + measInfoList_, ((bitField0_ & 0x00000010) == 0x00000010), + getParentForChildren(), isClean()); + measInfoList_ = null; + } + return measInfoListBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Measdatacollection) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Measdatacollection) + private static final PmProtoGenerated.PmRopFile.Measdatacollection DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Measdatacollection(); + } + + public static PmProtoGenerated.PmRopFile.Measdatacollection getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Measdatacollection parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Measdatacollection(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Measdatacollection getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface Perf3gppfieldsOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Perf3gppfields) + com.google.protobuf.MessageOrBuilder { + + /** + * string perf3gppFieldsVersion = 1; + */ + java.lang.String getPerf3GppFieldsVersion(); + + /** + * string perf3gppFieldsVersion = 1; + */ + com.google.protobuf.ByteString getPerf3GppFieldsVersionBytes(); + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + boolean hasMeasDataCollection(); + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + PmProtoGenerated.PmRopFile.Measdatacollection getMeasDataCollection(); + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + PmProtoGenerated.PmRopFile.MeasdatacollectionOrBuilder getMeasDataCollectionOrBuilder(); + } + + /** + * Protobuf type {@code PmRopFile.Perf3gppfields} + */ + public static final class Perf3gppfields extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Perf3gppfields) + Perf3gppfieldsOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Perf3gppfields.newBuilder() to construct. + private Perf3gppfields(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Perf3gppfields() { + perf3GppFieldsVersion_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Perf3gppfields(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + + perf3GppFieldsVersion_ = s; + break; + } + case 18: { + PmProtoGenerated.PmRopFile.Measdatacollection.Builder subBuilder = null; + if (measDataCollection_ != null) { + subBuilder = measDataCollection_.toBuilder(); + } + measDataCollection_ = input.readMessage( + PmProtoGenerated.PmRopFile.Measdatacollection.parser(), extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(measDataCollection_); + measDataCollection_ = subBuilder.buildPartial(); + } + + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Perf3gppfields_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Perf3gppfields_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Perf3gppfields.class, + PmProtoGenerated.PmRopFile.Perf3gppfields.Builder.class); + } + + public static final int PERF3GPPFIELDSVERSION_FIELD_NUMBER = 1; + private volatile java.lang.Object perf3GppFieldsVersion_; + + /** + * string perf3gppFieldsVersion = 1; + */ + public java.lang.String getPerf3GppFieldsVersion() { + java.lang.Object ref = perf3GppFieldsVersion_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + perf3GppFieldsVersion_ = s; + return s; + } + } + + /** + * string perf3gppFieldsVersion = 1; + */ + public com.google.protobuf.ByteString getPerf3GppFieldsVersionBytes() { + java.lang.Object ref = perf3GppFieldsVersion_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + perf3GppFieldsVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int MEASDATACOLLECTION_FIELD_NUMBER = 2; + private PmProtoGenerated.PmRopFile.Measdatacollection measDataCollection_; + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public boolean hasMeasDataCollection() { + return measDataCollection_ != null; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public PmProtoGenerated.PmRopFile.Measdatacollection getMeasDataCollection() { + return measDataCollection_ == null ? PmProtoGenerated.PmRopFile.Measdatacollection.getDefaultInstance() + : measDataCollection_; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public PmProtoGenerated.PmRopFile.MeasdatacollectionOrBuilder getMeasDataCollectionOrBuilder() { + return getMeasDataCollection(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (!getPerf3GppFieldsVersionBytes().isEmpty()) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, perf3GppFieldsVersion_); + } + if (measDataCollection_ != null) { + output.writeMessage(2, getMeasDataCollection()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (!getPerf3GppFieldsVersionBytes().isEmpty()) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, perf3GppFieldsVersion_); + } + if (measDataCollection_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(2, getMeasDataCollection()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Perf3gppfields)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Perf3gppfields other = (PmProtoGenerated.PmRopFile.Perf3gppfields) obj; + + boolean result = true; + result = result && getPerf3GppFieldsVersion().equals(other.getPerf3GppFieldsVersion()); + result = result && (hasMeasDataCollection() == other.hasMeasDataCollection()); + if (hasMeasDataCollection()) { + result = result && getMeasDataCollection().equals(other.getMeasDataCollection()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + PERF3GPPFIELDSVERSION_FIELD_NUMBER; + hash = (53 * hash) + getPerf3GppFieldsVersion().hashCode(); + if (hasMeasDataCollection()) { + hash = (37 * hash) + MEASDATACOLLECTION_FIELD_NUMBER; + hash = (53 * hash) + getMeasDataCollection().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom( + com.google.protobuf.CodedInputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Perf3gppfields prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Perf3gppfields} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Perf3gppfields) + PmProtoGenerated.PmRopFile.Perf3gppfieldsOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Perf3gppfields_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Perf3gppfields_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Perf3gppfields.class, + PmProtoGenerated.PmRopFile.Perf3gppfields.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Perf3gppfields.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + perf3GppFieldsVersion_ = ""; + + if (measDataCollectionBuilder_ == null) { + measDataCollection_ = null; + } else { + measDataCollection_ = null; + measDataCollectionBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Perf3gppfields_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Perf3gppfields getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Perf3gppfields.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Perf3gppfields build() { + PmProtoGenerated.PmRopFile.Perf3gppfields result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Perf3gppfields buildPartial() { + PmProtoGenerated.PmRopFile.Perf3gppfields result = + new PmProtoGenerated.PmRopFile.Perf3gppfields(this); + result.perf3GppFieldsVersion_ = perf3GppFieldsVersion_; + if (measDataCollectionBuilder_ == null) { + result.measDataCollection_ = measDataCollection_; + } else { + result.measDataCollection_ = measDataCollectionBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Perf3gppfields) { + return mergeFrom((PmProtoGenerated.PmRopFile.Perf3gppfields) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Perf3gppfields other) { + if (other == PmProtoGenerated.PmRopFile.Perf3gppfields.getDefaultInstance()) + return this; + if (!other.getPerf3GppFieldsVersion().isEmpty()) { + perf3GppFieldsVersion_ = other.perf3GppFieldsVersion_; + onChanged(); + } + if (other.hasMeasDataCollection()) { + mergeMeasDataCollection(other.getMeasDataCollection()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Perf3gppfields parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Perf3gppfields) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private java.lang.Object perf3GppFieldsVersion_ = ""; + + /** + * string perf3gppFieldsVersion = 1; + */ + public java.lang.String getPerf3GppFieldsVersion() { + java.lang.Object ref = perf3GppFieldsVersion_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + perf3GppFieldsVersion_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * string perf3gppFieldsVersion = 1; + */ + public com.google.protobuf.ByteString getPerf3GppFieldsVersionBytes() { + java.lang.Object ref = perf3GppFieldsVersion_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + perf3GppFieldsVersion_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * string perf3gppFieldsVersion = 1; + */ + public Builder setPerf3GppFieldsVersion(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + perf3GppFieldsVersion_ = value; + onChanged(); + return this; + } + + /** + * string perf3gppFieldsVersion = 1; + */ + public Builder clearPerf3GppFieldsVersion() { + + perf3GppFieldsVersion_ = getDefaultInstance().getPerf3GppFieldsVersion(); + onChanged(); + return this; + } + + /** + * string perf3gppFieldsVersion = 1; + */ + public Builder setPerf3GppFieldsVersionBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + perf3GppFieldsVersion_ = value; + onChanged(); + return this; + } + + private PmProtoGenerated.PmRopFile.Measdatacollection measDataCollection_ = null; + private com.google.protobuf.SingleFieldBuilderV3 measDataCollectionBuilder_; + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public boolean hasMeasDataCollection() { + return measDataCollectionBuilder_ != null || measDataCollection_ != null; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public PmProtoGenerated.PmRopFile.Measdatacollection getMeasDataCollection() { + if (measDataCollectionBuilder_ == null) { + return measDataCollection_ == null + ? PmProtoGenerated.PmRopFile.Measdatacollection.getDefaultInstance() + : measDataCollection_; + } else { + return measDataCollectionBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public Builder setMeasDataCollection(PmProtoGenerated.PmRopFile.Measdatacollection value) { + if (measDataCollectionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + measDataCollection_ = value; + onChanged(); + } else { + measDataCollectionBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public Builder setMeasDataCollection( + PmProtoGenerated.PmRopFile.Measdatacollection.Builder builderForValue) { + if (measDataCollectionBuilder_ == null) { + measDataCollection_ = builderForValue.build(); + onChanged(); + } else { + measDataCollectionBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public Builder mergeMeasDataCollection(PmProtoGenerated.PmRopFile.Measdatacollection value) { + if (measDataCollectionBuilder_ == null) { + if (measDataCollection_ != null) { + measDataCollection_ = PmProtoGenerated.PmRopFile.Measdatacollection + .newBuilder(measDataCollection_).mergeFrom(value).buildPartial(); + } else { + measDataCollection_ = value; + } + onChanged(); + } else { + measDataCollectionBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public Builder clearMeasDataCollection() { + if (measDataCollectionBuilder_ == null) { + measDataCollection_ = null; + onChanged(); + } else { + measDataCollection_ = null; + measDataCollectionBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public PmProtoGenerated.PmRopFile.Measdatacollection.Builder getMeasDataCollectionBuilder() { + + onChanged(); + return getMeasDataCollectionFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + public PmProtoGenerated.PmRopFile.MeasdatacollectionOrBuilder getMeasDataCollectionOrBuilder() { + if (measDataCollectionBuilder_ != null) { + return measDataCollectionBuilder_.getMessageOrBuilder(); + } else { + return measDataCollection_ == null + ? PmProtoGenerated.PmRopFile.Measdatacollection.getDefaultInstance() + : measDataCollection_; + } + } + + /** + * .PmRopFile.Measdatacollection measDataCollection = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3 getMeasDataCollectionFieldBuilder() { + if (measDataCollectionBuilder_ == null) { + measDataCollectionBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getMeasDataCollection(), getParentForChildren(), isClean()); + measDataCollection_ = null; + } + return measDataCollectionBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Perf3gppfields) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Perf3gppfields) + private static final PmProtoGenerated.PmRopFile.Perf3gppfields DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Perf3gppfields(); + } + + public static PmProtoGenerated.PmRopFile.Perf3gppfields getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Perf3gppfields parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Perf3gppfields(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Perf3gppfields getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface EventOrBuilder extends + // @@protoc_insertion_point(interface_extends:PmRopFile.Event) + com.google.protobuf.MessageOrBuilder { + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + boolean hasCommonEventHeader(); + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + PmProtoGenerated.PmRopFile.Commoneventheader getCommonEventHeader(); + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + PmProtoGenerated.PmRopFile.CommoneventheaderOrBuilder getCommonEventHeaderOrBuilder(); + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + boolean hasPerf3GppFields(); + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + PmProtoGenerated.PmRopFile.Perf3gppfields getPerf3GppFields(); + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + PmProtoGenerated.PmRopFile.Perf3gppfieldsOrBuilder getPerf3GppFieldsOrBuilder(); + } + + /** + * Protobuf type {@code PmRopFile.Event} + */ + public static final class Event extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:PmRopFile.Event) + EventOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Event.newBuilder() to construct. + private Event(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Event() {} + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Event(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + PmProtoGenerated.PmRopFile.Commoneventheader.Builder subBuilder = null; + if (commonEventHeader_ != null) { + subBuilder = commonEventHeader_.toBuilder(); + } + commonEventHeader_ = input.readMessage( + PmProtoGenerated.PmRopFile.Commoneventheader.parser(), extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(commonEventHeader_); + commonEventHeader_ = subBuilder.buildPartial(); + } + + break; + } + case 18: { + PmProtoGenerated.PmRopFile.Perf3gppfields.Builder subBuilder = null; + if (perf3GppFields_ != null) { + subBuilder = perf3GppFields_.toBuilder(); + } + perf3GppFields_ = input.readMessage(PmProtoGenerated.PmRopFile.Perf3gppfields.parser(), + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(perf3GppFields_); + perf3GppFields_ = subBuilder.buildPartial(); + } + + break; + } + default: { + if (!parseUnknownFieldProto3(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Event_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Event_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Event.class, + PmProtoGenerated.PmRopFile.Event.Builder.class); + } + + public static final int COMMONEVENTHEADER_FIELD_NUMBER = 1; + private PmProtoGenerated.PmRopFile.Commoneventheader commonEventHeader_; + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public boolean hasCommonEventHeader() { + return commonEventHeader_ != null; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public PmProtoGenerated.PmRopFile.Commoneventheader getCommonEventHeader() { + return commonEventHeader_ == null ? PmProtoGenerated.PmRopFile.Commoneventheader.getDefaultInstance() + : commonEventHeader_; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public PmProtoGenerated.PmRopFile.CommoneventheaderOrBuilder getCommonEventHeaderOrBuilder() { + return getCommonEventHeader(); + } + + public static final int PERF3GPPFIELDS_FIELD_NUMBER = 2; + private PmProtoGenerated.PmRopFile.Perf3gppfields perf3GppFields_; + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public boolean hasPerf3GppFields() { + return perf3GppFields_ != null; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public PmProtoGenerated.PmRopFile.Perf3gppfields getPerf3GppFields() { + return perf3GppFields_ == null ? PmProtoGenerated.PmRopFile.Perf3gppfields.getDefaultInstance() + : perf3GppFields_; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public PmProtoGenerated.PmRopFile.Perf3gppfieldsOrBuilder getPerf3GppFieldsOrBuilder() { + return getPerf3GppFields(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (commonEventHeader_ != null) { + output.writeMessage(1, getCommonEventHeader()); + } + if (perf3GppFields_ != null) { + output.writeMessage(2, getPerf3GppFields()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (commonEventHeader_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(1, getCommonEventHeader()); + } + if (perf3GppFields_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(2, getPerf3GppFields()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile.Event)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile.Event other = (PmProtoGenerated.PmRopFile.Event) obj; + + boolean result = true; + result = result && (hasCommonEventHeader() == other.hasCommonEventHeader()); + if (hasCommonEventHeader()) { + result = result && getCommonEventHeader().equals(other.getCommonEventHeader()); + } + result = result && (hasPerf3GppFields() == other.hasPerf3GppFields()); + if (hasPerf3GppFields()) { + result = result && getPerf3GppFields().equals(other.getPerf3GppFields()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasCommonEventHeader()) { + hash = (37 * hash) + COMMONEVENTHEADER_FIELD_NUMBER; + hash = (53 * hash) + getCommonEventHeader().hashCode(); + } + if (hasPerf3GppFields()) { + hash = (37 * hash) + PERF3GPPFIELDS_FIELD_NUMBER; + hash = (53 * hash) + getPerf3GppFields().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Event parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Event parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile.Event parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile.Event prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile.Event} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile.Event) + PmProtoGenerated.PmRopFile.EventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_Event_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_Event_fieldAccessorTable + .ensureFieldAccessorsInitialized(PmProtoGenerated.PmRopFile.Event.class, + PmProtoGenerated.PmRopFile.Event.Builder.class); + } + + // Construct using PmSchema.PmRopFile.Event.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (commonEventHeaderBuilder_ == null) { + commonEventHeader_ = null; + } else { + commonEventHeader_ = null; + commonEventHeaderBuilder_ = null; + } + if (perf3GppFieldsBuilder_ == null) { + perf3GppFields_ = null; + } else { + perf3GppFields_ = null; + perf3GppFieldsBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_Event_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Event getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.Event.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Event build() { + PmProtoGenerated.PmRopFile.Event result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Event buildPartial() { + PmProtoGenerated.PmRopFile.Event result = new PmProtoGenerated.PmRopFile.Event(this); + if (commonEventHeaderBuilder_ == null) { + result.commonEventHeader_ = commonEventHeader_; + } else { + result.commonEventHeader_ = commonEventHeaderBuilder_.build(); + } + if (perf3GppFieldsBuilder_ == null) { + result.perf3GppFields_ = perf3GppFields_; + } else { + result.perf3GppFields_ = perf3GppFieldsBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile.Event) { + return mergeFrom((PmProtoGenerated.PmRopFile.Event) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile.Event other) { + if (other == PmProtoGenerated.PmRopFile.Event.getDefaultInstance()) + return this; + if (other.hasCommonEventHeader()) { + mergeCommonEventHeader(other.getCommonEventHeader()); + } + if (other.hasPerf3GppFields()) { + mergePerf3GppFields(other.getPerf3GppFields()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile.Event parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile.Event) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private PmProtoGenerated.PmRopFile.Commoneventheader commonEventHeader_ = null; + private com.google.protobuf.SingleFieldBuilderV3 commonEventHeaderBuilder_; + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public boolean hasCommonEventHeader() { + return commonEventHeaderBuilder_ != null || commonEventHeader_ != null; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public PmProtoGenerated.PmRopFile.Commoneventheader getCommonEventHeader() { + if (commonEventHeaderBuilder_ == null) { + return commonEventHeader_ == null + ? PmProtoGenerated.PmRopFile.Commoneventheader.getDefaultInstance() + : commonEventHeader_; + } else { + return commonEventHeaderBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public Builder setCommonEventHeader(PmProtoGenerated.PmRopFile.Commoneventheader value) { + if (commonEventHeaderBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + commonEventHeader_ = value; + onChanged(); + } else { + commonEventHeaderBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public Builder setCommonEventHeader( + PmProtoGenerated.PmRopFile.Commoneventheader.Builder builderForValue) { + if (commonEventHeaderBuilder_ == null) { + commonEventHeader_ = builderForValue.build(); + onChanged(); + } else { + commonEventHeaderBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public Builder mergeCommonEventHeader(PmProtoGenerated.PmRopFile.Commoneventheader value) { + if (commonEventHeaderBuilder_ == null) { + if (commonEventHeader_ != null) { + commonEventHeader_ = PmProtoGenerated.PmRopFile.Commoneventheader + .newBuilder(commonEventHeader_).mergeFrom(value).buildPartial(); + } else { + commonEventHeader_ = value; + } + onChanged(); + } else { + commonEventHeaderBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public Builder clearCommonEventHeader() { + if (commonEventHeaderBuilder_ == null) { + commonEventHeader_ = null; + onChanged(); + } else { + commonEventHeader_ = null; + commonEventHeaderBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public PmProtoGenerated.PmRopFile.Commoneventheader.Builder getCommonEventHeaderBuilder() { + + onChanged(); + return getCommonEventHeaderFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + public PmProtoGenerated.PmRopFile.CommoneventheaderOrBuilder getCommonEventHeaderOrBuilder() { + if (commonEventHeaderBuilder_ != null) { + return commonEventHeaderBuilder_.getMessageOrBuilder(); + } else { + return commonEventHeader_ == null + ? PmProtoGenerated.PmRopFile.Commoneventheader.getDefaultInstance() + : commonEventHeader_; + } + } + + /** + * .PmRopFile.Commoneventheader commonEventHeader = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3 getCommonEventHeaderFieldBuilder() { + if (commonEventHeaderBuilder_ == null) { + commonEventHeaderBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getCommonEventHeader(), getParentForChildren(), isClean()); + commonEventHeader_ = null; + } + return commonEventHeaderBuilder_; + } + + private PmProtoGenerated.PmRopFile.Perf3gppfields perf3GppFields_ = null; + private com.google.protobuf.SingleFieldBuilderV3 perf3GppFieldsBuilder_; + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public boolean hasPerf3GppFields() { + return perf3GppFieldsBuilder_ != null || perf3GppFields_ != null; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public PmProtoGenerated.PmRopFile.Perf3gppfields getPerf3GppFields() { + if (perf3GppFieldsBuilder_ == null) { + return perf3GppFields_ == null ? PmProtoGenerated.PmRopFile.Perf3gppfields.getDefaultInstance() + : perf3GppFields_; + } else { + return perf3GppFieldsBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public Builder setPerf3GppFields(PmProtoGenerated.PmRopFile.Perf3gppfields value) { + if (perf3GppFieldsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + perf3GppFields_ = value; + onChanged(); + } else { + perf3GppFieldsBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public Builder setPerf3GppFields(PmProtoGenerated.PmRopFile.Perf3gppfields.Builder builderForValue) { + if (perf3GppFieldsBuilder_ == null) { + perf3GppFields_ = builderForValue.build(); + onChanged(); + } else { + perf3GppFieldsBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public Builder mergePerf3GppFields(PmProtoGenerated.PmRopFile.Perf3gppfields value) { + if (perf3GppFieldsBuilder_ == null) { + if (perf3GppFields_ != null) { + perf3GppFields_ = PmProtoGenerated.PmRopFile.Perf3gppfields.newBuilder(perf3GppFields_) + .mergeFrom(value).buildPartial(); + } else { + perf3GppFields_ = value; + } + onChanged(); + } else { + perf3GppFieldsBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public Builder clearPerf3GppFields() { + if (perf3GppFieldsBuilder_ == null) { + perf3GppFields_ = null; + onChanged(); + } else { + perf3GppFields_ = null; + perf3GppFieldsBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public PmProtoGenerated.PmRopFile.Perf3gppfields.Builder getPerf3GppFieldsBuilder() { + + onChanged(); + return getPerf3GppFieldsFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + public PmProtoGenerated.PmRopFile.Perf3gppfieldsOrBuilder getPerf3GppFieldsOrBuilder() { + if (perf3GppFieldsBuilder_ != null) { + return perf3GppFieldsBuilder_.getMessageOrBuilder(); + } else { + return perf3GppFields_ == null ? PmProtoGenerated.PmRopFile.Perf3gppfields.getDefaultInstance() + : perf3GppFields_; + } + } + + /** + * .PmRopFile.Perf3gppfields perf3gppFields = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3 getPerf3GppFieldsFieldBuilder() { + if (perf3GppFieldsBuilder_ == null) { + perf3GppFieldsBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getPerf3GppFields(), getParentForChildren(), isClean()); + perf3GppFields_ = null; + } + return perf3GppFieldsBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile.Event) + } + + // @@protoc_insertion_point(class_scope:PmRopFile.Event) + private static final PmProtoGenerated.PmRopFile.Event DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile.Event(); + } + + public static PmProtoGenerated.PmRopFile.Event getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Event parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Event(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile.Event getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public static final int EVENT_FIELD_NUMBER = 1; + private PmProtoGenerated.PmRopFile.Event event_; + + /** + * .PmRopFile.Event event = 1; + */ + public boolean hasEvent() { + return event_ != null; + } + + /** + * .PmRopFile.Event event = 1; + */ + public PmProtoGenerated.PmRopFile.Event getEvent() { + return event_ == null ? PmProtoGenerated.PmRopFile.Event.getDefaultInstance() : event_; + } + + /** + * .PmRopFile.Event event = 1; + */ + public PmProtoGenerated.PmRopFile.EventOrBuilder getEventOrBuilder() { + return getEvent(); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (event_ != null) { + output.writeMessage(1, getEvent()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (event_ != null) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(1, getEvent()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof PmProtoGenerated.PmRopFile)) { + return super.equals(obj); + } + PmProtoGenerated.PmRopFile other = (PmProtoGenerated.PmRopFile) obj; + + boolean result = true; + result = result && (hasEvent() == other.hasEvent()); + if (hasEvent()) { + result = result && getEvent().equals(other.getEvent()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasEvent()) { + hash = (37 * hash) + EVENT_FIELD_NUMBER; + hash = (53 * hash) + getEvent().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static PmProtoGenerated.PmRopFile parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static PmProtoGenerated.PmRopFile parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile parseFrom(java.io.InputStream input) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static PmProtoGenerated.PmRopFile parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static PmProtoGenerated.PmRopFile parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(PmProtoGenerated.PmRopFile prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code PmRopFile} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:PmRopFile) + PmProtoGenerated.PmRopFileOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return PmProtoGenerated.internal_static_PmRopFile_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return PmProtoGenerated.internal_static_PmRopFile_fieldAccessorTable.ensureFieldAccessorsInitialized( + PmProtoGenerated.PmRopFile.class, PmProtoGenerated.PmRopFile.Builder.class); + } + + // Construct using PmSchema.PmRopFile.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (eventBuilder_ == null) { + event_ = null; + } else { + event_ = null; + eventBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return PmProtoGenerated.internal_static_PmRopFile_descriptor; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile getDefaultInstanceForType() { + return PmProtoGenerated.PmRopFile.getDefaultInstance(); + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile build() { + PmProtoGenerated.PmRopFile result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile buildPartial() { + PmProtoGenerated.PmRopFile result = new PmProtoGenerated.PmRopFile(this); + if (eventBuilder_ == null) { + result.event_ = event_; + } else { + result.event_ = eventBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return (Builder) super.clone(); + } + + @java.lang.Override + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof PmProtoGenerated.PmRopFile) { + return mergeFrom((PmProtoGenerated.PmRopFile) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(PmProtoGenerated.PmRopFile other) { + if (other == PmProtoGenerated.PmRopFile.getDefaultInstance()) + return this; + if (other.hasEvent()) { + mergeEvent(other.getEvent()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { + PmProtoGenerated.PmRopFile parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (PmProtoGenerated.PmRopFile) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private PmProtoGenerated.PmRopFile.Event event_ = null; + private com.google.protobuf.SingleFieldBuilderV3 eventBuilder_; + + /** + * .PmRopFile.Event event = 1; + */ + public boolean hasEvent() { + return eventBuilder_ != null || event_ != null; + } + + /** + * .PmRopFile.Event event = 1; + */ + public PmProtoGenerated.PmRopFile.Event getEvent() { + if (eventBuilder_ == null) { + return event_ == null ? PmProtoGenerated.PmRopFile.Event.getDefaultInstance() : event_; + } else { + return eventBuilder_.getMessage(); + } + } + + /** + * .PmRopFile.Event event = 1; + */ + public Builder setEvent(PmProtoGenerated.PmRopFile.Event value) { + if (eventBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + event_ = value; + onChanged(); + } else { + eventBuilder_.setMessage(value); + } + + return this; + } + + /** + * .PmRopFile.Event event = 1; + */ + public Builder setEvent(PmProtoGenerated.PmRopFile.Event.Builder builderForValue) { + if (eventBuilder_ == null) { + event_ = builderForValue.build(); + onChanged(); + } else { + eventBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + + /** + * .PmRopFile.Event event = 1; + */ + public Builder mergeEvent(PmProtoGenerated.PmRopFile.Event value) { + if (eventBuilder_ == null) { + if (event_ != null) { + event_ = PmProtoGenerated.PmRopFile.Event.newBuilder(event_).mergeFrom(value).buildPartial(); + } else { + event_ = value; + } + onChanged(); + } else { + eventBuilder_.mergeFrom(value); + } + + return this; + } + + /** + * .PmRopFile.Event event = 1; + */ + public Builder clearEvent() { + if (eventBuilder_ == null) { + event_ = null; + onChanged(); + } else { + event_ = null; + eventBuilder_ = null; + } + + return this; + } + + /** + * .PmRopFile.Event event = 1; + */ + public PmProtoGenerated.PmRopFile.Event.Builder getEventBuilder() { + + onChanged(); + return getEventFieldBuilder().getBuilder(); + } + + /** + * .PmRopFile.Event event = 1; + */ + public PmProtoGenerated.PmRopFile.EventOrBuilder getEventOrBuilder() { + if (eventBuilder_ != null) { + return eventBuilder_.getMessageOrBuilder(); + } else { + return event_ == null ? PmProtoGenerated.PmRopFile.Event.getDefaultInstance() : event_; + } + } + + /** + * .PmRopFile.Event event = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3 getEventFieldBuilder() { + if (eventBuilder_ == null) { + eventBuilder_ = + new com.google.protobuf.SingleFieldBuilderV3( + getEvent(), getParentForChildren(), isClean()); + event_ = null; + } + return eventBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFieldsProto3(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PmRopFile) + } + + // @@protoc_insertion_point(class_scope:PmRopFile) + private static final PmProtoGenerated.PmRopFile DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new PmProtoGenerated.PmRopFile(); + } + + public static PmProtoGenerated.PmRopFile getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PmRopFile parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new PmRopFile(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public PmProtoGenerated.PmRopFile getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Commoneventheader_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Commoneventheader_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Measinfoid_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Measinfoid_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Meastypes_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Meastypes_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Measresults_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Measresults_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Measvalueslist_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Measvalueslist_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Measinfolist_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Measinfolist_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Measdatacollection_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Measdatacollection_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Perf3gppfields_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Perf3gppfields_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_PmRopFile_Event_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PmRopFile_Event_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + static { + java.lang.String[] descriptorData = {"\n\017pm_schema.proto\"\344\010\n\tPmRopFile\022\037\n\005event" + + "\030\001 \001(\0132\020.PmRopFile.Event\032\235\002\n\021Commonevent" + + "header\022\016\n\006domain\030\001 \001(\t\022\017\n\007eventId\030\002 \001(\t\022" + + "\020\n\010sequence\030\003 \001(\r\022\021\n\teventName\030\004 \001(\t\022\022\n\n" + + "sourceName\030\005 \001(\t\022\033\n\023reportingEntityName\030" + + "\006 \001(\t\022\020\n\010priority\030\007 \001(\t\022\032\n\022startEpochMic" + + "rosec\030\010 \001(\004\022\031\n\021lastEpochMicrosec\030\t \001(\004\022\017" + + "\n\007version\030\n \001(\t\022\037\n\027vesEventListenerVersi" + + "on\030\013 \001(\t\022\026\n\016timeZoneOffset\030\014 \001(\t\032!\n\nMeas" + + "infoid\022\023\n\013sMeasInfoId\030\001 \001(\t\032#\n\tMeastypes" + + "\022\026\n\016sMeasTypesList\030\001 \003(\t\032(\n\013Measresults\022" + + "\t\n\001p\030\001 \001(\r\022\016\n\006sValue\030\002 \001(\t\032i\n\016Measvalues" + + "list\022\025\n\rmeasObjInstId\030\001 \001(\t\022\023\n\013suspectFl" + + "ag\030\002 \001(\t\022+\n\013measResults\030\003 \003(\0132\026.PmRopFil" + + "e.Measresults\032\225\001\n\014Measinfolist\022)\n\nmeasIn" + + "foId\030\001 \001(\0132\025.PmRopFile.Measinfoid\022\'\n\tmea" + + "sTypes\030\002 \001(\0132\024.PmRopFile.Meastypes\0221\n\016me" + + "asValuesList\030\003 \003(\0132\031.PmRopFile.Measvalue" + + "slist\032\277\001\n\022Measdatacollection\022\031\n\021granular" + + "ityPeriod\030\001 \001(\r\022\036\n\026measuredEntityUserNam" + + "e\030\002 \001(\t\022\030\n\020measuredEntityDn\030\003 \001(\t\022%\n\035mea" + + "suredEntitySoftwareVersion\030\004 \001(\t\022-\n\014meas" + + "InfoList\030\005 \003(\0132\027.PmRopFile.Measinfolist\032" + + "j\n\016Perf3gppfields\022\035\n\025perf3gppFieldsVersi" + + "on\030\001 \001(\t\0229\n\022measDataCollection\030\002 \001(\0132\035.P" + + "mRopFile.Measdatacollection\032s\n\005Event\0227\n\021" + + "commonEventHeader\030\001 \001(\0132\034.PmRopFile.Comm" + + "oneventheader\0221\n\016perf3gppFields\030\002 \001(\0132\031." + + "PmRopFile.Perf3gppfieldsb\006proto3"}; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] {}, assigner); + internal_static_PmRopFile_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_PmRopFile_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_descriptor, new java.lang.String[] {"Event",}); + internal_static_PmRopFile_Commoneventheader_descriptor = + internal_static_PmRopFile_descriptor.getNestedTypes().get(0); + internal_static_PmRopFile_Commoneventheader_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Commoneventheader_descriptor, + new java.lang.String[] {"Domain", "EventId", "Sequence", "EventName", "SourceName", + "ReportingEntityName", "Priority", "StartEpochMicrosec", "LastEpochMicrosec", "Version", + "VesEventListenerVersion", "TimeZoneOffset",}); + internal_static_PmRopFile_Measinfoid_descriptor = internal_static_PmRopFile_descriptor.getNestedTypes().get(1); + internal_static_PmRopFile_Measinfoid_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Measinfoid_descriptor, new java.lang.String[] {"SMeasInfoId",}); + internal_static_PmRopFile_Meastypes_descriptor = internal_static_PmRopFile_descriptor.getNestedTypes().get(2); + internal_static_PmRopFile_Meastypes_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Meastypes_descriptor, new java.lang.String[] {"SMeasTypesList",}); + internal_static_PmRopFile_Measresults_descriptor = internal_static_PmRopFile_descriptor.getNestedTypes().get(3); + internal_static_PmRopFile_Measresults_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Measresults_descriptor, new java.lang.String[] {"P", "SValue",}); + internal_static_PmRopFile_Measvalueslist_descriptor = + internal_static_PmRopFile_descriptor.getNestedTypes().get(4); + internal_static_PmRopFile_Measvalueslist_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Measvalueslist_descriptor, + new java.lang.String[] {"MeasObjInstId", "SuspectFlag", "MeasResults",}); + internal_static_PmRopFile_Measinfolist_descriptor = + internal_static_PmRopFile_descriptor.getNestedTypes().get(5); + internal_static_PmRopFile_Measinfolist_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Measinfolist_descriptor, + new java.lang.String[] {"MeasInfoId", "MeasTypes", "MeasValuesList",}); + internal_static_PmRopFile_Measdatacollection_descriptor = + internal_static_PmRopFile_descriptor.getNestedTypes().get(6); + internal_static_PmRopFile_Measdatacollection_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Measdatacollection_descriptor, + new java.lang.String[] {"GranularityPeriod", "MeasuredEntityUserName", "MeasuredEntityDn", + "MeasuredEntitySoftwareVersion", "MeasInfoList",}); + internal_static_PmRopFile_Perf3gppfields_descriptor = + internal_static_PmRopFile_descriptor.getNestedTypes().get(7); + internal_static_PmRopFile_Perf3gppfields_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Perf3gppfields_descriptor, + new java.lang.String[] {"Perf3GppFieldsVersion", "MeasDataCollection",}); + internal_static_PmRopFile_Event_descriptor = internal_static_PmRopFile_descriptor.getNestedTypes().get(8); + internal_static_PmRopFile_Event_fieldAccessorTable = + new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_PmRopFile_Event_descriptor, + new java.lang.String[] {"CommonEventHeader", "Perf3GppFields",}); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/src/test/resources/A20000626.2315+0200-2330+0200_HTTPS-6-73.xml.gz101.json b/src/test/resources/A20000626.2315+0200-2330+0200_HTTPS-6-73.xml.gz101.json new file mode 100644 index 0000000..ef5f41a --- /dev/null +++ b/src/test/resources/A20000626.2315+0200-2330+0200_HTTPS-6-73.xml.gz101.json @@ -0,0 +1 @@ +{"event":{"commonEventHeader":{"domain":"","eventId":"","sequence":0,"eventName":"","sourceName":"HTTPS-6","reportingEntityName":"","priority":"","startEpochMicrosec":15198378,"lastEpochMicrosec":151983,"version":"","vesEventListenerVersion":"","timeZoneOffset":"UTC+05:00"},"perf3gppFields":{"perf3gppFieldsVersion":"1.0","measDataCollection":{"granularityPeriod":900,"measuredEntityUserName":"","measuredEntityDn":"seliitdus00487","measuredEntitySoftwareVersion":"CXP2010174_1 R44B17","measInfoList":[{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellCU_GNBCUCP"},"measTypes":{"sMeasTypesList":["pmAnrNcgiMeasFailUeCap","pmAnrNcgiMeasRcvDrx","pmAnrNcgiMeasRcvNoSib1NoTacCell","pmAnrNcgiMeasTriggerDrx","pmAnrNeighbRelAddNr","pmAnrNeighbRelRemNr","pmCaConfigAtt","pmCaConfigDlSamp","pmCaConfigDlSumEndcDistr","pmCaConfigDlSumSaDistr","pmCaConfigSucc","pmCaDeconfigAtt","pmCaDeconfigSucc","pmDrbEstabAttArp","pmDrbEstabFailIntgProt","pmDrbEstabSuccArp","pmDrbRelAbnormalAmfActArp","pmDrbRelAbnormalAmfArp","pmDrbRelAbnormalGnbActArp","pmDrbRelAbnormalGnbArp","pmDrbRelNormalArp","pmDrbRopEndEndc","pmDrbRopEndSa","pmEndcCaConfigAtt","pmEndcCaConfigSucc","pmEndcCaDeconfigAtt","pmEndcCaDeconfigSucc","pmEndcMnHoPSCellKeepAtt","pmEndcMnHoPSCellKeepSucc","pmEndcPSCellChangeRejInIntraGnbBbIntens","pmEndcPSCellChangeRejInIntraGnbMpLoad","pmEndcPSCellChangeResAllocReqInterSgnb","pmEndcPSCellChangeResAllocSuccInterSgnb","pmEndcSetupScgUeAtt","pmEndcSetupScgUeSucc","pmHoExeAttOutInterGnbTs","pmHoExeAttOutIntraGnbTs","pmHoExeSuccOutInterGnbTs","pmHoExeSuccOutIntraGnbTs","pmHoPrepAttOutInterGnbTs","pmHoPrepAttOutIntraGnbTs","pmHoPrepRejInInterGnbBbIntens","pmHoPrepRejInInterGnbMpLoad","pmHoPrepRejInIntraGnbBbIntens","pmHoPrepRejInIntraGnbMpLoad","pmHoPrepSuccOutInterGnbTs","pmHoPrepSuccOutIntraGnbTs","pmNgSigConnEstabAttEm","pmNgSigConnEstabSuccEm","pmNrdcRelRelocUeMnAtt","pmNrdcRelRelocUeMnSucc","pmNrdcRelUeSnAbnormalMnInit","pmNrdcRelUeSnAbnormalMnInitAct","pmNrdcRelUeSnAbnormalSnInit","pmNrdcRelUeSnAbnormalSnInitAct","pmNrdcRelUeSnNormal","pmNrdcSetupUeMnAtt","pmNrdcSetupUeMnSucc","pmNrdcSetupUeSnAtt","pmNrdcSetupUeSnSucc","pmPduSessionEstabAtt","pmPduSessionEstabAttIntgProt64kbps","pmPduSessionEstabSucc","pmPduSessionModifyAtt","pmPduSessionModifySucc","pmPduSessionRelAbnormalAmf","pmPduSessionRelAbnormalGnb","pmPduSessionRelNormal","pmPwsCancelReqCmas","pmPwsCancelReqEtws","pmPwsCancelRespCmas","pmPwsCancelRespEtws","pmPwsReqAttCmas","pmPwsReqAttEtwsPrimary","pmPwsReqAttEtwsSecondary","pmPwsReqSuccCmas","pmPwsReqSuccEtwsPrimary","pmPwsReqSuccEtwsSecondary","pmRrcConnEstabAttEm","pmRrcConnEstabAttReattEm","pmRrcConnEstabSuccEm","pmRrcConnLevelMaxNrDc","pmRrcConnLevelSumNrDc","pmRwrEutranUeNonPerfEpsfbEm","pmRwrEutranUeSuccEpsfb","pmRwrEutranUeSuccEpsfbEm","pmRwrEutranUeSuccEpsfbEmFbInd","pmRwrEutranUeSuccNrCoverage","pmRwrIntraNrUeSuccCoverage","pmUeCtxtRelAbnormalAmf","pmUeCtxtRelAbnormalGnb","pmUeCtxtRelNormal"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"180"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0"},{"p":92,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"180"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0"},{"p":92,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"180"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0"},{"p":92,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"180"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0"},{"p":92,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellDU_GNBDU"},"measTypes":{"sMeasTypesList":["pmDrxSleepTime","pmDrxTotalTime","pmDuCellDownLockAuto","pmDuCellDownLockMan","pmDuCellDownUnlockMan","pmMacBsrLcg0Distr","pmMacBsrLcg1Distr","pmMacBsrLcg2Distr","pmMacBsrLcg3Distr","pmMacBsrLcg4Distr","pmMacBsrLcg5Distr","pmMacBsrLcg6Distr","pmMacBsrLcg7Distr","pmMacContentionDelayDlDistr","pmMacContentionDelayDlMaxQos","pmMacContentionDelayUlDistr","pmMacHarqDlAck16Qam","pmMacHarqDlAck16QamInit","pmMacHarqDlAck256Qam","pmMacHarqDlAck256QamInit","pmMacHarqDlAck64Qam","pmMacHarqDlAck64QamInit","pmMacHarqDlAckQpsk","pmMacHarqDlAckQpskInit","pmMacHarqDlDtx16Qam","pmMacHarqDlDtx16QamInit","pmMacHarqDlDtx256Qam","pmMacHarqDlDtx256QamInit","pmMacHarqDlDtx64Qam","pmMacHarqDlDtx64QamInit","pmMacHarqDlDtxQpsk","pmMacHarqDlDtxQpskInit","pmMacHarqDlFail","pmMacHarqDlNack16Qam","pmMacHarqDlNack16QamInit","pmMacHarqDlNack256Qam","pmMacHarqDlNack256QamInit","pmMacHarqDlNack64Qam","pmMacHarqDlNack64QamInit","pmMacHarqDlNackQpsk","pmMacHarqDlNackQpskInit","pmMacHarqUlAck16Qam","pmMacHarqUlAck16QamInit","pmMacHarqUlAck256Qam","pmMacHarqUlAck256QamInit","pmMacHarqUlAck64Qam","pmMacHarqUlAck64QamInit","pmMacHarqUlAckQpsk","pmMacHarqUlAckQpskInit","pmMacHarqUlDtx16Qam","pmMacHarqUlDtx16QamInit","pmMacHarqUlDtx256Qam","pmMacHarqUlDtx256QamInit","pmMacHarqUlDtx64Qam","pmMacHarqUlDtx64QamInit","pmMacHarqUlDtxQpsk","pmMacHarqUlDtxQpskInit","pmMacHarqUlFail","pmMacHarqUlNack16Qam","pmMacHarqUlNack16QamInit","pmMacHarqUlNack256Qam","pmMacHarqUlNack256QamInit","pmMacHarqUlNack64Qam","pmMacHarqUlNack64QamInit","pmMacHarqUlNackQpsk","pmMacHarqUlNackQpskInit","pmMacOffloadVolDlSCell","pmMacOffloadVolDlSCellExt","pmMacPdcchBlockingPdschOccasions","pmMacPdcchBlockingPuschOccasions","pmMacPucchSrPuschGrantLatDistr","pmMacRBSymPucchSchedPusch","pmMacRBSymRachSchedPusch","pmMacRBSymUsedNrEfficiency","pmMacRBSymUsedPdcchTypeAAggrLvl1","pmMacRBSymUsedPdcchTypeAAggrLvl16","pmMacRBSymUsedPdcchTypeAAggrLvl2","pmMacRBSymUsedPdcchTypeAAggrLvl4","pmMacRBSymUsedPdcchTypeAAggrLvl8","pmMacRBSymUsedPdschAdvMimoUserDistr","pmMacRBSymUsedPdschAdvSuMimoLayerDistr","pmMacRBSymUsedPdschMimoLayerDistr","pmMacRBSymUsedPdschMimoUserDistr","pmMacRBSymUsedPuschMimoLayerDistr","pmMacRBSymUsedPuschMimoUserDistr","pmMacSchedSlotDlCellSchedDistr","pmMacSchedSlotDlSCellActivatedDistr","pmMacSchedSlotDlTempAlarmSCellDistr","pmMacSchedSlotUlCellSchedDistr","pmMacSchedSlotUlSCellActivatedDistr","pmMacSchedSlotUlTempAlarmSCellDistr","pmMacTempAlarmSCellDistr","pmMacThpUlUeMax","pmMacTimeDlDrbLastSlot","pmMacTimeDlDrbLastSlotQos","pmMacTimeDlDrbLastSlotSamp","pmMacTimeDlDrbLastSlotSampQos","pmMacTimeDlDrbSamp","pmMacTimeDlDrbSampQos","pmMacVolDlDrbLastSlot","pmMacVolDlDrbLastSlotQos","pmMacVolDlDrbSingleBurst","pmMacVolDlDrbSingleBurstSamp","pmMacVolDlSCell","pmMacVolDlSCellExt","pmMacVolUlResUeLastSlot","pmMacVolUlResUeLate","pmMacVolUlSCell","pmMacVolUlUnresUe","pmMacVolUlUnresUeBsrGrant","pmMacVolUlUnresUePreemptGrant","pmMacVolUlUnresUePucchSrGrant","pmMeasGapDurationDistr","pmMeasGapUeMaxDistr","pmMeasGapUeMnMaxDistr","pmMeasGapUeMnSumDistr","pmMeasGapUeSamp","pmMeasGapUeSumDistr","pmPagBundledUeDistr","pmPagBundledUeSampDistr","pmPagDiscarded","pmPagReceived","pmRadioBeamFailureRecoveryCurrentBeam","pmRadioBeamFailureRecoveryNewBeam","pmRadioBeamSwitchSucc","pmRadioBeamSwitchSuccMacCe","pmRadioBeamSwitchTargetNotAvailable","pmRadioMacCpofdmTransUl","pmRadioMacDftsTransUl","pmRadioMacWvfSwitchAttUlCpofdm","pmRadioMacWvfSwitchAttUlDfts","pmRadioMacWvfSwitchSuccUlCpofdm","pmRadioMacWvfSwitchSuccUlDfts","pmRadioMaxDeltaIpNDistr","pmRadioNarrowBeamAcqTimeDistr","pmRadioPathlossUlDistr","pmRadioPdschEmptySlots","pmRadioPdschEmptySlotsForced","pmRadioPdschTable1McsDistr","pmRadioPdschTable2McsDistr","pmRadioPucchFailSr","pmRadioPuschPhrDistr","pmRadioPuschTable1McsDistr","pmRadioPuschTable2McsDistr","pmRadioRaCbSuccMsg3KnownCrnti","pmRadioRecInterferenceNoOfEventsPrbDistr","pmRadioRecInterferencePowerDistr","pmRadioRecInterferencePwrSumPrbDistr","pmRadioServingBeamDlDistr","pmRadioSinrPucchDistr","pmRadioSinrPucchF0Distr","pmRadioSinrPucchF2Distr","pmRadioSinrPuschDistr","pmRadioSrsResAvailAntSwitching","pmRadioSrsResUsedAntSwitching","pmRadioSsbBeamSwitch","pmRlcArqDlAck","pmRlcArqDlNack","pmRlcArqUlAck","pmRlcArqUlNack","pmRlcDelayPktTransmitDl","pmRlcDelayPktTransmitDlQos","pmRlcDelayTimeDl","pmRlcDelayTimeDlQos"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":12,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":13,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":14,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"2880"},{"p":80,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":81,"sValue":"0,0,0,0,0,0,0,0"},{"p":82,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":83,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":84,"sValue":"0,0,0,0,0,0,0,0"},{"p":85,"sValue":"0,0,0,0,0,0,0,0"},{"p":86,"sValue":"0,0,0,0,0,0,0,0"},{"p":87,"sValue":"0,0,0,0,0,0,0,0"},{"p":88,"sValue":"0,0,0,0,0,0,0,0"},{"p":89,"sValue":"0,0,0,0,0,0,0,0"},{"p":90,"sValue":"0,0,0,0,0,0,0,0"},{"p":91,"sValue":"0,0,0,0,0,0,0,0"},{"p":92,"sValue":"0,0,0,0,0,0,0,0"},{"p":93,"sValue":"0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"179"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0,0,0,0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"900,0,0,0,0"},{"p":135,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":136,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":137,"sValue":"1440001"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":140,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":143,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":144,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":147,"sValue":"0,0,0,0,0,0,0,0,0,0,0,1800001,0,0,0,0,0,0,0,0,0"},{"p":148,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":149,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":150,"sValue":"0,0,0,0,0,0,0,0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0"},{"p":155,"sValue":"0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0"},{"p":159,"sValue":"0"},{"p":160,"sValue":"0"},{"p":161,"sValue":"0"},{"p":162,"sValue":"0"},{"p":163,"sValue":"0"},{"p":164,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":12,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":13,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":14,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"5376"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":81,"sValue":"0,0,0,0,0,0,0,0"},{"p":82,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":83,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":84,"sValue":"0,0,0,0,0,0,0,0"},{"p":85,"sValue":"0,0,0,0,0,0,0,0"},{"p":86,"sValue":"0,0,0,0,0,0,0,0"},{"p":87,"sValue":"0,0,0,0,0,0,0,0"},{"p":88,"sValue":"0,0,0,0,0,0,0,0"},{"p":89,"sValue":"0,0,0,0,0,0,0,0"},{"p":90,"sValue":"0,0,0,0,0,0,0,0"},{"p":91,"sValue":"0,0,0,0,0,0,0,0"},{"p":92,"sValue":"0,0,0,0,0,0,0,0"},{"p":93,"sValue":"0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"179"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0,0,0,0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"900,0,0,0,0"},{"p":135,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":136,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":137,"sValue":"1440001"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":140,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":143,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":144,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":147,"sValue":"0,0,0,0,0,0,0,0,0,0,0,1800001,0,0,0,0,0,0,0,0,0"},{"p":148,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":149,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":150,"sValue":"0,0,0,0,0,0,0,0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0"},{"p":155,"sValue":"0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0"},{"p":159,"sValue":"0"},{"p":160,"sValue":"0"},{"p":161,"sValue":"0"},{"p":162,"sValue":"0"},{"p":163,"sValue":"0"},{"p":164,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":12,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":13,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":14,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":81,"sValue":"0,0,0,0,0,0,0,0"},{"p":82,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":83,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":84,"sValue":"0,0,0,0,0,0,0,0"},{"p":85,"sValue":"0,0,0,0,0,0,0,0"},{"p":86,"sValue":"0,0,0,0,0,0,0,0"},{"p":87,"sValue":"0,0,0,0,0,0,0,0"},{"p":88,"sValue":"0,0,0,0,0,0,0,0"},{"p":89,"sValue":"0,0,0,0,0,0,0,0"},{"p":90,"sValue":"0,0,0,0,0,0,0,0"},{"p":91,"sValue":"0,0,0,0,0,0,0,0"},{"p":92,"sValue":"0,0,0,0,0,0,0,0"},{"p":93,"sValue":"0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"0"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0,0,0,0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"0,0,0,0,0"},{"p":135,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":136,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":137,"sValue":"0"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":140,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":143,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":144,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":147,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":148,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":149,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":150,"sValue":"0,0,0,0,0,0,0,0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0"},{"p":155,"sValue":"0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0"},{"p":159,"sValue":"0"},{"p":160,"sValue":"0"},{"p":161,"sValue":"0"},{"p":162,"sValue":"0"},{"p":163,"sValue":"0"},{"p":164,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":12,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":13,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":14,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":81,"sValue":"0,0,0,0,0,0,0,0"},{"p":82,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":83,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":84,"sValue":"0,0,0,0,0,0,0,0"},{"p":85,"sValue":"0,0,0,0,0,0,0,0"},{"p":86,"sValue":"0,0,0,0,0,0,0,0"},{"p":87,"sValue":"0,0,0,0,0,0,0,0"},{"p":88,"sValue":"0,0,0,0,0,0,0,0"},{"p":89,"sValue":"0,0,0,0,0,0,0,0"},{"p":90,"sValue":"0,0,0,0,0,0,0,0"},{"p":91,"sValue":"0,0,0,0,0,0,0,0"},{"p":92,"sValue":"0,0,0,0,0,0,0,0"},{"p":93,"sValue":"0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"0"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0,0,0,0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"0,0,0,0,0"},{"p":135,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":136,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":137,"sValue":"0"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":140,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":143,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":144,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":147,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":148,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":149,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":150,"sValue":"0,0,0,0,0,0,0,0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0"},{"p":155,"sValue":"0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0"},{"p":159,"sValue":"0"},{"p":160,"sValue":"0"},{"p":161,"sValue":"0"},{"p":162,"sValue":"0"},{"p":163,"sValue":"0"},{"p":164,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRSectorCarrier_GNBDU"},"measTypes":{"sMeasTypesList":["pmBwAssignedDistr","pmEmfPwrBackoffDynResThrDistr","pmEmfPwrBackoffHardLimit","pmEmfPwrBackoffOnPwrDistr","pmEmfPwrBackoffPwrDistr","pmEmfPwrBackoffSamp","pmEmfPwrBackoffStepPwrDistr","pmEmfPwrBackoffSum","pmEmfPwrBackoffSumOn","pmRadioPowerDlDistr","pmRadioScaledTxPowerMax","pmSectorCarrierDowntime","pmSectorCarrierDowntimeAuto","pmSectorCarrierDowntimeAutoCbrs","pmSectorCarrierDowntimeMan","pmSectorCarrierDowntimeManCbrs"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRSectorCarrier=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,15,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"10000"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRSectorCarrier=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,15,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"15000"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRSectorCarrier=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"15,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRSectorCarrier=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"15,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EUtranCellFDD"},"measTypes":{"sMeasTypesList":["pmRadioHomUeRepRankDistr","pmRadioTxRankDistr","pmRadioUeRepRankDistr","pmRadioUeRepTm4Rank1PmiDistr","pmRadioUeRepTm4Rank2PmiDistr","pmRadioUeRepTm4Rank3PmiDistr","pmRadioUeRepTm4Rank4PmiDistr"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellDU_GNBDU"},"measTypes":{"sMeasTypesList":["pmMacRBSymUsedPdschAdvSuMimoTxRankDistr","pmRadioSchedUlSuMimoRankDistr","pmRadioUeRepCqi256QamRank1Distr","pmRadioUeRepCqi256QamRank2Distr","pmRadioUeRepCqi256QamRank3Distr","pmRadioUeRepCqi256QamRank4Distr","pmRadioUeRepCqi64QamRank1Distr","pmRadioUeRepCqi64QamRank2Distr","pmRadioUeRepCqi64QamRank3Distr","pmRadioUeRepCqi64QamRank4Distr","pmRadioUeRepPmiPrimaryCsiRsRank1Distr","pmRadioUeRepPmiPrimaryCsiRsRank2Distr","pmRadioUeRepPmiPrimaryCsiRsRank3Distr","pmRadioUeRepPmiPrimaryCsiRsRank4Distr","pmRadioUeRepPmiSecondaryCsiRsRank1Distr","pmRadioUeRepPmiSecondaryCsiRsRank2Distr","pmRadioUeRepPmiSecondaryCsiRsRank3Distr","pmRadioUeRepPmiSecondaryCsiRsRank4Distr","pmRadioUeRepRankDistr"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0"},{"p":3,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0"},{"p":3,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0"},{"p":3,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0"},{"p":3,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":4,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0,0,0,0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EUtranCellFDD"},"measTypes":{"sMeasTypesList":["pmRadioRecInterferencePwrPucch"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellDU_GNBDU"},"measTypes":{"sMeasTypesList":["pmRadioRecInterferencePwrPucchF0Distr","pmRadioRecInterferencePwrPucchF2Distr"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=GNBCUCPFunction_GNBCUCP"},"measTypes":{"sMeasTypesList":["pmEndcInitAccessFailMpOverload","pmRrcConnLevelMaxEnDc","pmRrcConnLevelMaxSa","pmRrcConnLevelSamp","pmRrcConnLevelSumEnDc","pmRrcConnLevelSumSa"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"180"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellCU_GNBCUCP"},"measTypes":{"sMeasTypesList":["pmCellDowntimeAuto","pmCellDowntimeMan","pmDrbEstabAtt5qi","pmDrbEstabSucc5qi","pmDrbLevelMaxEndc","pmDrbLevelMaxSa","pmDrbLevelSamp","pmDrbLevelSumEndc","pmDrbLevelSumSa","pmDrbRelAbnormalAmf5qi","pmDrbRelAbnormalAmfAct5qi","pmDrbRelAbnormalGnb5qi","pmDrbRelAbnormalGnbAct5qi","pmDrbRelNormal5qi","pmEndcPSCellChangeAttInterSgnb","pmEndcPSCellChangeAttIntraSgnb","pmEndcPSCellChangeSuccInterSgnb","pmEndcPSCellChangeSuccIntraSgnb","pmEndcRelUeAbnormalMenb","pmEndcRelUeAbnormalMenbAct","pmEndcRelUeAbnormalSgnb","pmEndcRelUeAbnormalSgnbAct","pmEndcRelUeNormal","pmEndcSetupUeAtt","pmEndcSetupUeSucc","pmHoExeAttOutEutran","pmHoExeAttOutInterGnb","pmHoExeAttOutIntraGnb","pmHoExeSuccOutEutran","pmHoExeSuccOutInterGnb","pmHoExeSuccOutIntraGnb","pmHoPrepAttOutEutran","pmHoPrepAttOutInterGnb","pmHoPrepAttOutIntraGnb","pmHoPrepSuccOutEutran","pmHoPrepSuccOutInterGnb","pmHoPrepSuccOutIntraGnb","pmNgSigConnEstabAtt","pmNgSigConnEstabAttMos","pmNgSigConnEstabSucc","pmNgSigConnEstabSuccMos","pmPduSessionLevelMax","pmPduSessionLevelSamp","pmPduSessionLevelSum","pmRrcConnEstabAtt","pmRrcConnEstabAttMos","pmRrcConnEstabAttReatt","pmRrcConnEstabAttReattMos","pmRrcConnEstabSucc","pmRrcConnEstabSuccMos","pmRrcConnLevelMaxEnDc","pmRrcConnLevelMaxSa","pmRrcConnLevelSamp","pmRrcConnLevelSumEnDc","pmRrcConnLevelSumSa","pmSessionTimeDrb5qi","pmUeCtxtEstabAtt","pmUeCtxtEstabSucc"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"180"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"180"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"180"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"180"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"180"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"180"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"180"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"180"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"180"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"180"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"180"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"180"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=GNBDUFunction_GNBDU"},"measTypes":{"sMeasTypesList":["pmLic10MHzBwHwacFdd","pmLic10MHzBwHwacTdd","pmLic25Plus25MHzScActual","pmLic5MHzBwHwacFdd","pmLic5MHzBwHwacTdd","pmLic5Plus5MHzScActual","pmLic5Plus5MHzScFdd","pmLic5Plus5MHzScTdd","pmLicDlCapDistr","pmLicDlCapUsedMax","pmLicDlCapUsedSamp","pmLicDlCapUsedSum","pmLicLteNrDynSpectrumSharingActual","pmLicLteNrDynSpectrumSharingFdd","pmLicLteNrSpectrumSharingFdd","pmLicUlCapDistr","pmLicUlCapUsedMax","pmLicUlCapUsedSamp","pmLicUlCapUsedSum"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"32"},{"p":6,"sValue":"16"},{"p":7,"sValue":"0"},{"p":8,"sValue":"16"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellDU_GNBDU"},"measTypes":{"sMeasTypesList":["pmActiveUeDlMax","pmActiveUeDlSamp","pmActiveUeDlSum","pmActiveUeDlTrueSamp","pmActiveUeUlMax","pmActiveUeUlSamp","pmActiveUeUlSum","pmActiveUeUlTrueSamp","pmCellDlSlots","pmCellDowntimeAuto","pmCellDowntimeMan","pmCellDowntimeManual","pmMacLatTimeDlDrxSyncQos","pmMacLatTimeDlDrxSyncSampQos","pmMacLatTimeDlNoDrxSyncQos","pmMacLatTimeDlNoDrxSyncSampQos","pmMacRBSymAvailDl","pmMacRBSymAvailPucch","pmMacRBSymAvailRach","pmMacRBSymAvailUl","pmMacRBSymCsiRs","pmMacRBSymUsedPdcchTypeA","pmMacRBSymUsedPdcchTypeB","pmMacRBSymUsedPdschTypeA","pmMacRBSymUsedPdschTypeABroadcasting","pmMacRBSymUsedPuschTypeA","pmMacRBSymUsedPuschTypeB","pmMacRBSymUtilDlDistr","pmMacRBSymUtilDlMax","pmMacRBSymUtilUlDistr","pmMacRBSymUtilUlMax","pmMacTimeDlDrb","pmMacTimeDlDrbQos","pmMacTimeUlResUe","pmMacVolDl","pmMacVolDlDrb","pmMacVolDlDrbQos","pmMacVolUl","pmMacVolUlResUe","pmPdschAvailTime","pmPdschSchedActivity","pmPdschSchedEntityDistr","pmPuschAvailTime","pmPuschSchedActivity","pmPuschSchedEntityDistr","pmRadioRaAttTaDistr","pmRadioRaCbAttMsg2","pmRadioRaCbFailMsg1MaxMsg3Sched","pmRadioRaCbFailMsg1Ooc","pmRadioRaCbFailMsg2Disc","pmRadioRaCbFailMsg3Crc","pmRadioRaCbFailMsg3Crnti","pmRadioRaCbFailMsg3Dtx","pmRadioRaCbPreambles","pmRadioRaCbSuccMsg3","pmRadioThpVolDl","pmRadioThpVolUl","pmRrcAccessFailBbIntens","pmRrcAccessFailMpLoad","pmSchedActivityCellDl","pmSchedActivityCellUl","pmUeCtxtAllocatedMax","pmUeCtxtAllocatedSamp","pmUeCtxtAllocatedSum","pmUeCtxtSetupAtt","pmUeCtxtSetupSucc"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"1800000"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"1800001"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"3032642268"},{"p":18,"sValue":"20160056"},{"p":19,"sValue":"15120000"},{"p":20,"sValue":"816482268"},{"p":21,"sValue":"0"},{"p":22,"sValue":"2880"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"3780540"},{"p":26,"sValue":"2520"},{"p":27,"sValue":"0"},{"p":28,"sValue":"900,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"900,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"5760004"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"1440004"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":46,"sValue":"2,1,1,2,2,3,2,1,1,0,0,0"},{"p":47,"sValue":"15"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"60"},{"p":54,"sValue":"15"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"179"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"1800000"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"1800001"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"5110563822"},{"p":18,"sValue":"20160056"},{"p":19,"sValue":"15120000"},{"p":20,"sValue":"1375923822"},{"p":21,"sValue":"0"},{"p":22,"sValue":"5376"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"3780504"},{"p":26,"sValue":"2352"},{"p":27,"sValue":"0"},{"p":28,"sValue":"900,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"900,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"5760004"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"1440004"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":46,"sValue":"3,1,2,1,2,0,3,2,0,0,0,0"},{"p":47,"sValue":"14"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"56"},{"p":54,"sValue":"14"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"179"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"900"},{"p":12,"sValue":"900"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":46,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBDUFunction=1,NRCellDU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"900"},{"p":12,"sValue":"900"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":46,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=PpControlLink_GNBCUUP"},"measTypes":{"sMeasTypesList":["pmDrbLevelMaxEndc","pmDrbLevelMaxSa","pmDrbLevelSamp","pmDrbLevelSumEndc","pmDrbLevelSumSa","pmPduSessionLevelMax","pmPduSessionLevelSamp","pmPduSessionLevelSum"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUUPFunction=1,PpControlTermination=1,PpControlLink=internal","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"900"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"900"},{"p":8,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EUtranCellFDD"},"measTypes":{"sMeasTypesList":["pmDlAssigsDetHarqMcPttNDO","pmDlAssigsDetHarqNonMcPttNDO","pmDlAssigsDetectedHarqNDO","pmDlAssigsDetectedHarqVolteNDO","pmDlAssigsWithDetHarqAckMcPtt","pmDlAssigsWithDetHarqAckNonMcPtt","pmDlAssigsWithDetectedHarqAck","pmDlAssigsWithDetectedHarqAckPCell","pmDlAssigsWithDetectedHarqAckSCell","pmDlAssigsWithDetectedHarqAckVolte","pmMacHarqDlAck16qam","pmMacHarqDlAck256qam","pmMacHarqDlAck64qam","pmMacHarqDlAckComp","pmMacHarqDlAckQpsk","pmMacHarqDlDtx16qam","pmMacHarqDlDtx256qam","pmMacHarqDlDtx64qam","pmMacHarqDlDtxQpsk","pmMacHarqDlNack16qam","pmMacHarqDlNack256qam","pmMacHarqDlNack64qam","pmMacHarqDlNackComp","pmMacHarqDlNackQpsk","pmMacHarqFail","pmMacHarqUlDtx16qam","pmMacHarqUlDtx256Qam","pmMacHarqUlDtx64Qam","pmMacHarqUlDtxQpsk","pmMacHarqUlFail16qam","pmMacHarqUlFail256Qam","pmMacHarqUlFail64Qam","pmMacHarqUlFailIua16qam","pmMacHarqUlFailIuaQpsk","pmMacHarqUlFailQpsk","pmMacHarqUlSucc16qam","pmMacHarqUlSucc256Qam","pmMacHarqUlSucc64Qam","pmMacHarqUlSuccQpsk","pmRaAccHarqFailAfterContRes","pmRaAccHarqFailWithinContRes","pmRaRejHarqFailAfterContRes","pmRaRejHarqFailWithinContRes","pmRlcArqDlAck","pmRlcArqDlNack","pmRlcArqRetxDl","pmRlcArqUlAck","pmRlcArqUlNack"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=FieldReplaceableUnit"},"measTypes":{"sMeasTypesList":["pmPowerFailure","pmUnitTemperatureLevel"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"4,4,4"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"4,4,4"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"4,4,4"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":" , , "}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1","suspectFlag":"false","measResults":[{"p":1,"sValue":" "},{"p":2,"sValue":"4,4,4"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=RiEthernetPort"},"measTypes":{"sMeasTypesList":["ifHCInBroadcastPkts","ifHCInMulticastPkts","ifHCInOctets","ifHCInUcastPkts","ifHCOutBroadcastPkts","ifHCOutMulticastPkts","ifHCOutOctets","ifHCOutUcastPkts","ifInErrors","ifOutErrors"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=P,RiEthernetPort=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"51885"},{"p":3,"sValue":"4628315"},{"p":4,"sValue":"9190"},{"p":5,"sValue":"0"},{"p":6,"sValue":"15493"},{"p":7,"sValue":"1956503"},{"p":8,"sValue":"7326"},{"p":9,"sValue":"0"},{"p":10,"sValue":" "}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=N,RiEthernetPort=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"37309"},{"p":3,"sValue":"2868954"},{"p":4,"sValue":"4551"},{"p":5,"sValue":"0"},{"p":6,"sValue":"915"},{"p":7,"sValue":"380361"},{"p":8,"sValue":"4549"},{"p":9,"sValue":"0"},{"p":10,"sValue":" "}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=B,RiEthernetPort=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"15490"},{"p":3,"sValue":"1512626"},{"p":4,"sValue":"7326"},{"p":5,"sValue":"0"},{"p":6,"sValue":"51883"},{"p":7,"sValue":"3492085"},{"p":8,"sValue":"9190"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=A,RiEthernetPort=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"916"},{"p":3,"sValue":"260228"},{"p":4,"sValue":"4549"},{"p":5,"sValue":"0"},{"p":6,"sValue":"37309"},{"p":7,"sValue":"2097258"},{"p":8,"sValue":"4551"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=RiLink"},"measTypes":{"sMeasTypesList":["pmLinkDelayHigh","pmLinkDelayLow","pmLinkRestart","pmLinkStabilityError"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,RiLink=S3-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"1077"},{"p":2,"sValue":"1076"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,RiLink=S2-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"450"},{"p":2,"sValue":"450"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,RiLink=S1-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"427"},{"p":2,"sValue":"427"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,RiLink=R608-2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"2147483647"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,RiLink=R608-1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"2147483647"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=RiPort"},"measTypes":{"sMeasTypesList":["pmBitError","pmLossOfFrame","pmLossOfSignal"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,RiPort=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,RiPort=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,RiPort=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,RiPort=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,RiPort=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,RiPort=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=P","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=N","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=B","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,RiPort=A","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=K","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=C","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=B","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,RiPort=A","suspectFlag":"false","measResults":[{"p":1,"sValue":"0,0,0,0,0,0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=SfpChannel"},"measTypes":{"sMeasTypesList":["pmChannelRxPower","pmChannelTxBias","pmChannelTxPower"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=P,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2951"},{"p":2,"sValue":"3600"},{"p":3,"sValue":"2922"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=N,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"3018"},{"p":2,"sValue":"4200"},{"p":3,"sValue":"2953"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=K,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2809"},{"p":2,"sValue":"3500"},{"p":3,"sValue":"2812"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,SfpModule=DATA_2,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,SfpModule=DATA_2,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,SfpModule=DATA_2,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,SfpModule=DATA_1,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2810"},{"p":2,"sValue":"3100"},{"p":3,"sValue":"2830"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,SfpModule=DATA_1,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2717"},{"p":2,"sValue":"3600"},{"p":3,"sValue":"2775"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,SfpModule=DATA_1,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2700"},{"p":2,"sValue":"3900"},{"p":3,"sValue":"2773"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=C,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=B,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2744"},{"p":2,"sValue":"3200"},{"p":3,"sValue":"2775"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=B,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2863"},{"p":2,"sValue":"3800"},{"p":3,"sValue":"2898"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=A,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2684"},{"p":2,"sValue":"3900"},{"p":3,"sValue":"2737"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=A,SfpChannel=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"2892"},{"p":2,"sValue":"4400"},{"p":3,"sValue":"2893"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=SfpModule"},"measTypes":{"sMeasTypesList":["pmSfpTemperature","pmSfpVoltage"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,SfpModule=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,SfpModule=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"1310"},{"p":2,"sValue":"32552"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,SfpModule=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,SfpModule=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"1430"},{"p":2,"sValue":"32904"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,SfpModule=DATA_2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,SfpModule=DATA_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"1410"},{"p":2,"sValue":"33141"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=P","suspectFlag":"false","measResults":[{"p":1,"sValue":"1388"},{"p":2,"sValue":"32558"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=N","suspectFlag":"false","measResults":[{"p":1,"sValue":"1384"},{"p":2,"sValue":"32505"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=B","suspectFlag":"false","measResults":[{"p":1,"sValue":"1297"},{"p":2,"sValue":"32999"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,SfpModule=A","suspectFlag":"false","measResults":[{"p":1,"sValue":"1297"},{"p":2,"sValue":"33041"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=K","suspectFlag":"false","measResults":[{"p":1,"sValue":"1474"},{"p":2,"sValue":"32588"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=C","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=B","suspectFlag":"false","measResults":[{"p":1,"sValue":"1463"},{"p":2,"sValue":"32493"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,SfpModule=A","suspectFlag":"false","measResults":[{"p":1,"sValue":"1464"},{"p":2,"sValue":"32534"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EthernetPort"},"measTypes":{"sMeasTypesList":["ifHCInBroadcastPkts","ifHCInMulticastPkts","ifHCInOctets","ifHCInUcastPkts","ifHCOutBroadcastPkts","ifHCOutMulticastPkts","ifHCOutOctets","ifHCOutUcastPkts","ifInDiscards","ifInErrors","ifInUnknownProtos","ifInUnknownTags","ifOutDiscards","ifOutErrors"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,EthernetPort=TN_IDL_B_1","suspectFlag":"false","measResults":[{"p":1,"sValue":"22345"},{"p":2,"sValue":"1057"},{"p":3,"sValue":"2921772"},{"p":4,"sValue":"4472"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"9147991"},{"p":8,"sValue":"17099"},{"p":9,"sValue":"0"},{"p":10,"sValue":"7627"},{"p":11,"sValue":"3582"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=InterfaceIPv4"},"measTypes":{"sMeasTypesList":["ipIfStatsHCInOctets","ipIfStatsHCInReceives","ipIfStatsHCOutOctets","ipIfStatsHCOutTransmits","ipIfStatsInAddrErrors","ipIfStatsInDiscards","ipIfStatsInHdrErrors","ipIfStatsInNoRoutes","ipIfStatsInTruncatedPkts","ipIfStatsInUnknownProtos"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,Router=vr_OAM,InterfaceIPv4=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"314834"},{"p":2,"sValue":"4561"},{"p":3,"sValue":"8697703"},{"p":4,"sValue":"16811"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,Router=vr_NR,InterfaceIPv4=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"19900"},{"p":2,"sValue":"388"},{"p":3,"sValue":"1260"},{"p":4,"sValue":"30"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,Router=vr_LTE,InterfaceIPv4=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"26656"},{"p":2,"sValue":"574"},{"p":3,"sValue":"14580"},{"p":4,"sValue":"243"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,Router=Node_Internal_F1,InterfaceIPv4=NRDU","suspectFlag":"false","measResults":[{"p":1,"sValue":" "},{"p":2,"sValue":" "},{"p":3,"sValue":" "},{"p":4,"sValue":" "},{"p":5,"sValue":" "},{"p":6,"sValue":" "},{"p":7,"sValue":" "},{"p":8,"sValue":" "},{"p":9,"sValue":" "},{"p":10,"sValue":" "}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,Router=Node_Internal_F1,InterfaceIPv4=NRCUCP","suspectFlag":"false","measResults":[{"p":1,"sValue":" "},{"p":2,"sValue":" "},{"p":3,"sValue":" "},{"p":4,"sValue":" "},{"p":5,"sValue":" "},{"p":6,"sValue":" "},{"p":7,"sValue":" "},{"p":8,"sValue":" "},{"p":9,"sValue":" "},{"p":10,"sValue":" "}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=SctpAssociation"},"measTypes":{"sMeasTypesList":["sctpAssocInAbnormalControlChunks","sctpAssocInAbnormalDataChunks","sctpAssocInControlChunks","sctpAssocInDataChunks","sctpAssocInDiscardedControlChunks","sctpAssocInDiscardedDataChunks","sctpAssocInOctets","sctpAssocOutDataChunks","sctpAssocOutDiscardedDataChunks","sctpAssocOutDiscardedUserMsgs","sctpAssocOutOctets","sctpAssocRtxChunks"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,SctpEndpoint=F1_NRDU,SctpAssociation=38472-10.0.0.1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"833"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"87820"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"87820"},{"p":12,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,SctpEndpoint=F1_NRCUCP,SctpAssociation=38472-10.0.0.2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"833"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"87820"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"87820"},{"p":12,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=VlanPort"},"measTypes":{"sMeasTypesList":["ifHCInBroadcastPkts","ifHCInMulticastPkts","ifHCInOctets","ifHCInUcastPkts","ifHCOutBroadcastPkts","ifHCOutMulticastPkts","ifHCOutOctets","ifHCOutUcastPkts","ifOutDiscards"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,VlanPort=vr_OAM.IF1","suspectFlag":"false","measResults":[{"p":1,"sValue":"1996"},{"p":2,"sValue":"341"},{"p":3,"sValue":"657544"},{"p":4,"sValue":"4205"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"9067777"},{"p":8,"sValue":"16814"},{"p":9,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,VlanPort=vr_NR.IF1","suspectFlag":"false","measResults":[{"p":1,"sValue":"8457"},{"p":2,"sValue":"343"},{"p":3,"sValue":"711892"},{"p":4,"sValue":"33"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"2368"},{"p":8,"sValue":"37"},{"p":9,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Transport=1,VlanPort=vr_LTE.IF1","suspectFlag":"false","measResults":[{"p":1,"sValue":"11894"},{"p":2,"sValue":"343"},{"p":3,"sValue":"1006906"},{"p":4,"sValue":"234"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"20246"},{"p":8,"sValue":"248"},{"p":9,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=ConsumedEnergyMeasurement"},"measTypes":{"sMeasTypesList":["pmConsumedEnergy","pmConsumedEnergyAccumulated","pmPowerConsumption"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,NodeSupport=1,ConsumedEnergyMeasurement=1","suspectFlag":"true","measResults":[{"p":1,"sValue":" "},{"p":2,"sValue":" "},{"p":3,"sValue":" , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , "}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EnergyMeter"},"measTypes":{"sMeasTypesList":["pmConsumedEnergy","pmConsumedEnergyAccumulated","pmPowerConsumption","pmVoltage"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,EnergyMeter=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"83"},{"p":2,"sValue":"34846"},{"p":3,"sValue":"333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,334,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333,333"},{"p":4,"sValue":"52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S1-1,EnergyMeter=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"12"},{"p":2,"sValue":"4947"},{"p":3,"sValue":"47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47,47"},{"p":4,"sValue":"53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S2-1,EnergyMeter=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"13"},{"p":2,"sValue":"5394"},{"p":3,"sValue":"52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,51,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,51,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52,52"},{"p":4,"sValue":"54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54,54"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=Radio-S3-1,EnergyMeter=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"22"},{"p":2,"sValue":"9338"},{"p":3,"sValue":"89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89,89"},{"p":4,"sValue":"53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,52,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,52,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53,53"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=R608,EnergyMeter=1","suspectFlag":"true","measResults":[{"p":1,"sValue":" "},{"p":2,"sValue":" "},{"p":3,"sValue":" , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , "},{"p":4,"sValue":" , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , "}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=SupportUnit"},"measTypes":{"sMeasTypesList":["pmFanSpeed"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,SupportUnit=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"18,18,18"}]},{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,SupportUnit=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"30,30,30"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=BbProcessingResource"},"measTypes":{"sMeasTypesList":["pmLicDlCapActual","pmLicDlCapDistr","pmLicDlCapUsedMax","pmLicDlCapUsedSamp","pmLicDlCapUsedSum","pmLicDlPrbCapActual","pmLicDlPrbCapDistr","pmLicDlPrbUsedDistr","pmLicDlPrbUsedMax","pmLicDlPrbUsedSamp","pmLicDlPrbUsedSum","pmLicPrbUnit","pmLicUlCapActual","pmLicUlCapDistr","pmLicUlCapUsedMax","pmLicUlCapUsedSamp","pmLicUlCapUsedSum","pmLicUlPrbCapActual","pmLicUlPrbCapDistr","pmLicUlPrbUsedDistr","pmLicUlPrbUsedMax","pmLicUlPrbUsedSamp","pmLicUlPrbUsedSum","pmPdcpPktDiscDlEth","pmPdcpPktDiscUlEthPacing"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,BbProcessingResource=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":20,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=ENodeBFunction"},"measTypes":{"sMeasTypesList":["pmLic10MHzBwHwacFddAas","pmLic10MHzBwHwacTddAas","pmLic5MHzBwHwacNonAas","pmLic5MHzSectorCarrierActual","pmLic5Plus5MHzScFddActual","pmLic5Plus5MHzScTddActual","pmLicConnectedUsersActual","pmLicConnectedUsersDistr","pmLicConnectedUsersLevSamp","pmLicConnectedUsersLevSum","pmLicConnectedUsersLicense","pmLicConnectedUsersMax","pmLicConnectedUsersTimeCong","pmLicDlCapLicense","pmLicDlPrbCapLicense","pmLicUlCapLicense","pmLicUlPrbCapLicense","pmMoFootprintMax","pmObsLocalEventDiscPmLoad","pmRrcConnBrEnbLevSamp","pmRrcConnBrEnbLevSum","pmRrcConnBrEnbMax"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"5"},{"p":8,"sValue":"900,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"180"},{"p":10,"sValue":"0"},{"p":11,"sValue":"8000"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"10000"},{"p":15,"sValue":"1800"},{"p":16,"sValue":"10000"},{"p":17,"sValue":"1800"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EUtranCellFDD"},"measTypes":{"sMeasTypesList":["pmCellDowntimeAuto","pmCellDowntimeMan","pmCellHoExeAttLteInterF","pmCellHoExeAttLteIntraF","pmCellHoExeSuccLteInterF","pmCellHoExeSuccLteIntraF","pmCellHoPrepAttLteInterF","pmCellHoPrepAttLteIntraF","pmCellHoPrepSuccLteInterF","pmCellHoPrepSuccLteIntraF","pmDrbThpTimeDlQci","pmEndcSetupScgUeAtt","pmEndcSetupScgUeSucc","pmEndcSetupUeAtt","pmEndcSetupUeSucc","pmErabEstabAttAdded","pmErabEstabAttAddedArp","pmErabEstabAttAddedCsfb","pmErabEstabAttAddedCsfbArp","pmErabEstabAttAddedCsfbQci","pmErabEstabAttAddedHoOngoing","pmErabEstabAttAddedHoOngoingArp","pmErabEstabAttAddedHoOngoingQci","pmErabEstabAttAddedQci","pmErabEstabAttInit","pmErabEstabAttInitArp","pmErabEstabFailAddedLic","pmErabEstabFailAddedOngoingProc","pmErabEstabFailGbrDlEnb","pmErabEstabFailGbrUlEnb","pmErabEstabFailInitLic","pmErabEstabSuccAdded","pmErabEstabSuccAddedArp","pmErabEstabSuccAddedQci","pmErabEstabSuccInit","pmErabEstabSuccInitArp","pmErabRelAbnormalEnbAct","pmErabRelAbnormalMmeAct","pmErabRelMmeAct","pmErabRelNormalEnbAct","pmErabRelNormalEnbArp","pmEranCompUlAvoidedNack","pmEranCompUlNack","pmHoPrepRejInLicConnUsers","pmHoPrepRejInLicMob","pmHoPrepRejInSl","pmLcgThpTimeUlLcg","pmLcgThpVolUlLcg","pmMacCellThpTimeDl","pmMacCellThpTimeUl","pmMacDrbThpTimeDlQci","pmMacDrbThpVolDlQci","pmMacLcgThpTimeUlLcg","pmMacLcgThpVolUlLcg","pmMacTimeUlResUe","pmMacUeThpTimeDl","pmMacUeThpTimeUl","pmMacUeThpUlMbbHighVolDistr","pmMacUeThpUlMbbLowVolDistr","pmMacUeThpVolDl","pmMacUeThpVolUl","pmMacVolUlResUe","pmObsLocalEventDiscPmLoad","pmPagReceivedCe","pmPdcpLatPktTransDl","pmPdcpLatTimeDl","pmPdcpPktDiscDlPelr","pmPdcpPktDiscDlPelrUu","pmPdcpPktLostUl","pmPdcpPktReceivedUl","pmPdcpPktTransDl","pmPdcpUeThpTimeDl","pmPdcpVolDlDrb","pmPdcpVolDlDrbFiltQci","pmPdcpVolDlDrbLastTTI","pmPdcpVolDlDrbLastTTIQci","pmPdcpVolDlSrb","pmPdcpVolUlDrb","pmPdcpVolUlSrb","pmRrcConnEstabAtt","pmRrcConnEstabAttCe","pmRrcConnEstabAttDta","pmRrcConnEstabAttDtaCe","pmRrcConnEstabAttReatt","pmRrcConnEstabAttReattCe","pmRrcConnEstabAttReattDta","pmRrcConnEstabAttReattDtaCe","pmRrcConnEstabFailLic","pmRrcConnEstabFailLicActiveUsers","pmRrcConnEstabFailMmeOvlMod","pmRrcConnEstabFailMmeOvlModCe","pmRrcConnEstabFailMmeOvlMos","pmRrcConnEstabFailMmeOvlMosCe","pmRrcConnEstabSucc","pmRrcConnEstabSuccCe","pmRrcConnEstabSuccDta","pmRrcConnEstabSuccDtaCe","pmRrcConnEstabSuccGummeiNative","pmRrcConnResumeAtt","pmRrcConnResumeFallbackAtt","pmRrcConnResumeFallbackSucc","pmRrcConnResumeSucc","pmS1SigConnEstabAtt","pmS1SigConnEstabAttCe","pmS1SigConnEstabAttDta","pmS1SigConnEstabAttDtaCe","pmS1SigConnEstabAttEm","pmS1SigConnEstabAttHpa","pmS1SigConnEstabAttMod","pmS1SigConnEstabAttModCe","pmS1SigConnEstabAttMos","pmS1SigConnEstabAttMta","pmS1SigConnEstabAttMtaCe","pmS1SigConnEstabFailMmeOvlMos","pmS1SigConnEstabFailMmeOvlMosCe","pmS1SigConnEstabSucc","pmS1SigConnEstabSuccCe","pmS1SigConnEstabSuccDta","pmS1SigConnEstabSuccDtaCe","pmS1SigConnEstabSuccEm","pmS1SigConnEstabSuccHpa","pmS1SigConnEstabSuccMod","pmS1SigConnEstabSuccModCe","pmS1SigConnEstabSuccMos","pmS1SigConnEstabSuccMta","pmS1SigConnEstabSuccMtaCe","pmSessionTimeDrb","pmSessionTimeUe","pmUeCtxtConnResumeAtt","pmUeCtxtConnResumeSucc","pmUeCtxtRelAbnormalEnbAct","pmUeCtxtRelAbnormalMmeAct","pmUeCtxtRelCsfbCdma1xRtt","pmUeCtxtRelCsfbCdma1xRttEm","pmUeCtxtRelCsfbGsm","pmUeCtxtRelCsfbGsmEm","pmUeCtxtRelCsfbLimitedDualRadioUeCdma","pmUeCtxtRelCsfbTdScdma","pmUeCtxtRelCsfbTdScdmaEm","pmUeCtxtRelCsfbWcdma","pmUeCtxtRelCsfbWcdmaEm","pmUeCtxtRelMmeAct","pmUeCtxtResumeAtt","pmUeCtxtResumeSucc","pmUeExclThpDlShortDrb","pmUeExclThpDlShortDrb2","pmUeExclThpUlRelativeHighVol","pmUeExclThpUlRelativeLowVol","pmUeExclThpUlShortTimeHighVol","pmUeExclThpUlShortTimeLowVol","pmUeThp2DlDistr","pmUeThp2UlDistr","pmUeThpDlDistr","pmUeThpDlMbbFiltered2Distr","pmUeThpDlMbbFilteredDistr","pmUeThpTimeDl","pmUeThpTimeUl","pmUeThpUlDistr","pmUeThpVolUl"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"900"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0,0,0,0"},{"p":48,"sValue":"0,0,0,0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0,0,0,0"},{"p":54,"sValue":"0,0,0,0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":59,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0,0,0,0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0,0,0,0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0,0,0,0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0,0,0,0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0,0,0,0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0,0,0,0"},{"p":92,"sValue":"0"},{"p":93,"sValue":"0,0,0,0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0,0,0,0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0,0,0,0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0,0,0,0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0,0,0,0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0,0,0,0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0,0,0,0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"0,0,0,0"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0,0,0,0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0,0,0,0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"0"},{"p":135,"sValue":"0"},{"p":136,"sValue":"0"},{"p":137,"sValue":"0"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0"},{"p":140,"sValue":"0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0"},{"p":143,"sValue":"0"},{"p":144,"sValue":"0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0"},{"p":147,"sValue":"0"},{"p":148,"sValue":"0"},{"p":149,"sValue":"0"},{"p":150,"sValue":"0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":155,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0,0,0,0,0,0,0,0"},{"p":159,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"900"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0,0,0,0"},{"p":48,"sValue":"0,0,0,0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0,0,0,0"},{"p":54,"sValue":"0,0,0,0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":59,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0,0,0,0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"},{"p":74,"sValue":"0"},{"p":75,"sValue":"0"},{"p":76,"sValue":"0"},{"p":77,"sValue":"0"},{"p":78,"sValue":"0"},{"p":79,"sValue":"0"},{"p":80,"sValue":"0"},{"p":81,"sValue":"0,0,0,0"},{"p":82,"sValue":"0"},{"p":83,"sValue":"0,0,0,0"},{"p":84,"sValue":"0"},{"p":85,"sValue":"0,0,0,0"},{"p":86,"sValue":"0"},{"p":87,"sValue":"0,0,0,0"},{"p":88,"sValue":"0"},{"p":89,"sValue":"0"},{"p":90,"sValue":"0"},{"p":91,"sValue":"0,0,0,0"},{"p":92,"sValue":"0"},{"p":93,"sValue":"0,0,0,0"},{"p":94,"sValue":"0"},{"p":95,"sValue":"0,0,0,0"},{"p":96,"sValue":"0"},{"p":97,"sValue":"0,0,0,0"},{"p":98,"sValue":"0"},{"p":99,"sValue":"0"},{"p":100,"sValue":"0"},{"p":101,"sValue":"0"},{"p":102,"sValue":"0"},{"p":103,"sValue":"0"},{"p":104,"sValue":"0,0,0,0"},{"p":105,"sValue":"0"},{"p":106,"sValue":"0,0,0,0"},{"p":107,"sValue":"0"},{"p":108,"sValue":"0"},{"p":109,"sValue":"0"},{"p":110,"sValue":"0,0,0,0"},{"p":111,"sValue":"0"},{"p":112,"sValue":"0"},{"p":113,"sValue":"0,0,0,0"},{"p":114,"sValue":"0"},{"p":115,"sValue":"0,0,0,0"},{"p":116,"sValue":"0"},{"p":117,"sValue":"0,0,0,0"},{"p":118,"sValue":"0"},{"p":119,"sValue":"0,0,0,0"},{"p":120,"sValue":"0"},{"p":121,"sValue":"0"},{"p":122,"sValue":"0"},{"p":123,"sValue":"0,0,0,0"},{"p":124,"sValue":"0"},{"p":125,"sValue":"0"},{"p":126,"sValue":"0,0,0,0"},{"p":127,"sValue":"0"},{"p":128,"sValue":"0"},{"p":129,"sValue":"0"},{"p":130,"sValue":"0"},{"p":131,"sValue":"0"},{"p":132,"sValue":"0"},{"p":133,"sValue":"0"},{"p":134,"sValue":"0"},{"p":135,"sValue":"0"},{"p":136,"sValue":"0"},{"p":137,"sValue":"0"},{"p":138,"sValue":"0"},{"p":139,"sValue":"0"},{"p":140,"sValue":"0"},{"p":141,"sValue":"0"},{"p":142,"sValue":"0"},{"p":143,"sValue":"0"},{"p":144,"sValue":"0"},{"p":145,"sValue":"0"},{"p":146,"sValue":"0"},{"p":147,"sValue":"0"},{"p":148,"sValue":"0"},{"p":149,"sValue":"0"},{"p":150,"sValue":"0"},{"p":151,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":152,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":153,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":154,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":155,"sValue":"0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0"},{"p":156,"sValue":"0"},{"p":157,"sValue":"0"},{"p":158,"sValue":"0,0,0,0,0,0,0,0"},{"p":159,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=BbProcessingResource"},"measTypes":{"sMeasTypesList":["pmPdcpPktDiscDlEthBurst","pmPdcpPktDiscDlEthFormat","pmPdcpPktDiscDlEthX2Fwd","pmPdcpPktDiscDlNoMbmsCtxt","pmPdcpPktDiscDlNoUeCtxt"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,Equipment=1,FieldReplaceableUnit=BB-1,BbProcessingResource=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=EUtranCellFDD"},"measTypes":{"sMeasTypesList":["pmPdcpBitrateDlDrbMax","pmPdcpBitrateDlDrbMin","pmPdcpBitrateUlDrbMax","pmPdcpBitrateUlDrbMin","pmPdcpInactSecDlMcPttDistr","pmPdcpInactSecDlNonMcPttDistr","pmPdcpInactSecDlVolteDistr","pmPdcpInactSecUlMcPttDistr","pmPdcpInactSecUlNonMcPttDistr","pmPdcpInactSecUlVolteDistr","pmPdcpLatPktTransDlCatMDrxNoSyncQci","pmPdcpLatPktTransDlCatMDrxSyncQci","pmPdcpLatPktTransDlCatMNoDrxNoSyncQci","pmPdcpLatPktTransDlCatMNoDrxSyncQci","pmPdcpLatPktTransDlDrxNoSyncQci","pmPdcpLatPktTransDlDrxSync","pmPdcpLatPktTransDlDrxSyncQci","pmPdcpLatPktTransDlNoDrxNoSyncQci","pmPdcpLatPktTransDlNoDrxSyncQci","pmPdcpLatPktTransDlQci","pmPdcpLatTimeDlCatMDrxNoSyncQci","pmPdcpLatTimeDlCatMDrxSyncQci","pmPdcpLatTimeDlCatMNoDrxNoSyncQci","pmPdcpLatTimeDlCatMNoDrxSyncQci","pmPdcpLatTimeDlDrxNoSyncQci","pmPdcpLatTimeDlDrxSync","pmPdcpLatTimeDlDrxSyncQci","pmPdcpLatTimeDlNoDrxNoSyncQci","pmPdcpLatTimeDlNoDrxSyncQci","pmPdcpLatTimeDlQci","pmPdcpPktDiscDlAqm","pmPdcpPktDiscDlAqmQci","pmPdcpPktDiscDlHo","pmPdcpPktDiscDlHoQci","pmPdcpPktDiscDlPelrQci","pmPdcpPktDiscDlPelrUuQci","pmPdcpPktFwdDl","pmPdcpPktLostUlLimitations","pmPdcpPktLostUlMissingPdus2Qci","pmPdcpPktLostUlQci","pmPdcpPktLostUlRohcFail2Qci","pmPdcpPktLostUlSrbTooLarge","pmPdcpPktPdbUlOkMcPtt","pmPdcpPktPdbUlOkNonMcPtt","pmPdcpPktPdbUlOkVoip","pmPdcpPktReceivedDl","pmPdcpPktReceivedDlQci","pmPdcpPktReceivedUlQci","pmPdcpPktTransDlQci","pmPdcpVolDlCmpHdrQci","pmPdcpVolDlDrbCa","pmPdcpVolDlDrbLastTTICa","pmPdcpVolDlDrbQci","pmPdcpVolDlDrbTransPlmn0","pmPdcpVolDlDrbTransPlmn1","pmPdcpVolDlDrbTransPlmn2","pmPdcpVolDlDrbTransPlmn3","pmPdcpVolDlDrbTransPlmn4","pmPdcpVolDlDrbTransPlmn5","pmPdcpVolDlDrbTransPlmn6","pmPdcpVolDlDrbTransQci","pmPdcpVolDlHdrQci","pmPdcpVolDlSrbTrans","pmPdcpVolUlCmpHdrQci","pmPdcpVolUlDrbPlmn0","pmPdcpVolUlDrbPlmn1","pmPdcpVolUlDrbPlmn2","pmPdcpVolUlDrbPlmn3","pmPdcpVolUlDrbPlmn4","pmPdcpVolUlDrbPlmn5","pmPdcpVolUlDrbPlmn6","pmPdcpVolUlDrbQci","pmPdcpVolUlHdrQci"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,ENodeBFunction=1,EUtranCellFDD=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":6,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":7,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":8,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":9,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":10,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":11,"sValue":"0"},{"p":12,"sValue":"0"},{"p":13,"sValue":"0"},{"p":14,"sValue":"0"},{"p":15,"sValue":"0"},{"p":16,"sValue":"0"},{"p":17,"sValue":"0"},{"p":18,"sValue":"0"},{"p":19,"sValue":"0"},{"p":20,"sValue":"0"},{"p":21,"sValue":"0"},{"p":22,"sValue":"0"},{"p":23,"sValue":"0"},{"p":24,"sValue":"0"},{"p":25,"sValue":"0"},{"p":26,"sValue":"0"},{"p":27,"sValue":"0"},{"p":28,"sValue":"0"},{"p":29,"sValue":"0"},{"p":30,"sValue":"0"},{"p":31,"sValue":"0"},{"p":32,"sValue":"0"},{"p":33,"sValue":"0"},{"p":34,"sValue":"0"},{"p":35,"sValue":"0"},{"p":36,"sValue":"0"},{"p":37,"sValue":"0"},{"p":38,"sValue":"0"},{"p":39,"sValue":"0"},{"p":40,"sValue":"0"},{"p":41,"sValue":"0"},{"p":42,"sValue":"0,0,0,0,0,0,0,0,0,0"},{"p":43,"sValue":"0"},{"p":44,"sValue":"0"},{"p":45,"sValue":"0"},{"p":46,"sValue":"0"},{"p":47,"sValue":"0"},{"p":48,"sValue":"0"},{"p":49,"sValue":"0"},{"p":50,"sValue":"0"},{"p":51,"sValue":"0"},{"p":52,"sValue":"0"},{"p":53,"sValue":"0"},{"p":54,"sValue":"0"},{"p":55,"sValue":"0"},{"p":56,"sValue":"0"},{"p":57,"sValue":"0"},{"p":58,"sValue":"0"},{"p":59,"sValue":"0"},{"p":60,"sValue":"0"},{"p":61,"sValue":"0"},{"p":62,"sValue":"0"},{"p":63,"sValue":"0"},{"p":64,"sValue":"0"},{"p":65,"sValue":"0"},{"p":66,"sValue":"0"},{"p":67,"sValue":"0"},{"p":68,"sValue":"0"},{"p":69,"sValue":"0"},{"p":70,"sValue":"0"},{"p":71,"sValue":"0"},{"p":72,"sValue":"0"},{"p":73,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=GNBCUUPFunction_GNBCUUP"},"measTypes":{"sMeasTypesList":["pmPdcpPktDisc","pmPdcpPktDiscGtpu","pmPdcpPktDiscNoCtxt","pmPdcpPktFwdRecDlDiscQos","pmPdcpPktFwdRecDlQos","pmPdcpPktInjected","pmPdcpPktRecDlDiscInact","pmPdcpPktRecDlInact","pmPdcpRadioResSwitchDlMcg","pmPdcpRadioResSwitchDlScg"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUUPFunction=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"},{"p":3,"sValue":"0"},{"p":4,"sValue":"0"},{"p":5,"sValue":"0"},{"p":6,"sValue":"0"},{"p":7,"sValue":"0"},{"p":8,"sValue":"0"},{"p":9,"sValue":"0"},{"p":10,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=NRCellCU_GNBCUCP"},"measTypes":{"sMeasTypesList":["pmEndcUeCapabilityUlPdcpDelay"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=32","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=31","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=2","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"}]},{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUCPFunction=1,NRCellCU=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"}]}]},{"measInfoId":{"sMeasInfoId":"PM=1,PmGroup=X2UTermination_GNBCUUP"},"measTypes":{"sMeasTypesList":["pmPdcpPktFwdRecDlDiscQos","pmPdcpPktFwdRecDlQos"]},"measValuesList":[{"measObjInstId":"ManagedElement=seliitdus00487,GNBCUUPFunction=1,X2UTermination=1","suspectFlag":"false","measResults":[{"p":1,"sValue":"0"},{"p":2,"sValue":"0"}]}]}]}}}} \ No newline at end of file diff --git a/src/test/resources/pm_schema.proto b/src/test/resources/pm_schema.proto new file mode 100644 index 0000000..cebea10 --- /dev/null +++ b/src/test/resources/pm_schema.proto @@ -0,0 +1,64 @@ +syntax = "proto3"; + +message PmRopFile { + + message Commoneventheader { + string domain = 1; + string eventId = 2; + uint32 sequence = 3; + string eventName = 4; + string sourceName = 5; + string reportingEntityName = 6; + string priority = 7; + uint64 startEpochMicrosec = 8; + uint64 lastEpochMicrosec = 9; + string version = 10; + string vesEventListenerVersion = 11; + string timeZoneOffset = 12; + } + + message Measinfoid { + string sMeasInfoId = 1; + } + + message Meastypes { + repeated string sMeasTypesList = 1; + } + + message Measresults { + uint32 p = 1; + string sValue = 2; + } + + message Measvalueslist { + string measObjInstId = 1; + string suspectFlag = 2; + repeated Measresults measResults = 3; + } + + message Measinfolist { + Measinfoid measInfoId = 1; + Meastypes measTypes = 2; + repeated Measvalueslist measValuesList = 3; + } + + message Measdatacollection { + uint32 granularityPeriod = 1; + string measuredEntityUserName = 2; + string measuredEntityDn = 3; + string measuredEntitySoftwareVersion = 4; + repeated Measinfolist measInfoList = 5; + } + + message Perf3gppfields { + string perf3gppFieldsVersion = 1; + Measdatacollection measDataCollection = 2; + } + + message Event { + Commoneventheader commonEventHeader = 1; + Perf3gppfields perf3gppFields = 2; + } + + Event event = 1; +}