NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

NIFI-2854: Incorporated PR review feedback

NIFI-2854: Implemented feedback from PR Review

NIFI-2854: Ensure that all resources are closed on CompressableRecordReader.close() even if an IOException is thrown when closing one of them

This closes #1202
This commit is contained in:
Mark Payne 2016-10-04 09:38:14 -04:00 committed by Oleg Zhurakousky
parent 5a25884f52
commit 1be0871473
85 changed files with 7690 additions and 2103 deletions

View File

@ -31,6 +31,7 @@ public class NamedSearchableField implements SearchableField {
private final SearchableFieldType fieldType;
private final String friendlyName;
private final boolean attribute;
private final int hash; // cached for more efficient/faster use in sets and maps
NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
this(identifier, searchableName, friendlyName, attribute, SearchableFieldType.STRING);
@ -42,6 +43,7 @@ public class NamedSearchableField implements SearchableField {
this.friendlyName = requireNonNull(friendlyName);
this.attribute = requireNonNull(attribute);
this.fieldType = requireNonNull(fieldType);
this.hash = 298347 + searchableName.hashCode() + (attribute ? 1 : 0);
}
@Override
@ -76,11 +78,14 @@ public class NamedSearchableField implements SearchableField {
@Override
public int hashCode() {
return 298347 + searchableName.hashCode() + (attribute ? 1 : 0);
return hash;
}
@Override
public boolean equals(final Object obj) {
if (obj == this) {
return true;
}
if (obj == null) {
return false;
}
@ -90,6 +95,6 @@ public class NamedSearchableField implements SearchableField {
}
final SearchableField other = (SearchableField) obj;
return (this.searchableName.equals(other.getSearchableFieldName()) && attribute == other.isAttribute());
return attribute == other.isAttribute() && this.searchableName.equals(other.getSearchableFieldName());
}
}

View File

@ -104,6 +104,10 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
sourceQueueIdentifier = builder.sourceQueueIdentifier;
if (builder.eventId != null) {
eventId = builder.eventId;
}
}
public String getStorageFilename() {
@ -160,6 +164,14 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
return allAttrs;
}
public String getAttribute(final String attributeName) {
if (updatedAttributes.containsKey(attributeName)) {
return updatedAttributes.get(attributeName);
}
return previousAttributes.get(attributeName);
}
@Override
public String getComponentId() {
return componentId;
@ -417,6 +429,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
private long storageByteOffset = -1L;
private long eventDuration = -1L;
private String storageFilename;
private Long eventId;
private String contentClaimSection;
private String contentClaimContainer;
@ -478,6 +491,11 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
return this;
}
public Builder setEventId(final long eventId) {
this.eventId = eventId;
return this;
}
@Override
public ProvenanceEventBuilder copy() {
final Builder copy = new Builder();
@ -540,6 +558,16 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
return this;
}
public Builder setPreviousAttributes(final Map<String, String> previousAttributes) {
this.previousAttributes = previousAttributes;
return this;
}
public Builder setUpdatedAttributes(final Map<String, String> updatedAttributes) {
this.updatedAttributes = updatedAttributes;
return this;
}
@Override
public Builder setFlowFileUUID(final String uuid) {
this.uuid = uuid;
@ -649,6 +677,16 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
return this;
}
public Builder setChildUuids(final List<String> uuids) {
this.childrenUuids = uuids;
return this;
}
public Builder setParentUuids(final List<String> uuids) {
this.parentUuids = uuids;
return this;
}
@Override
public Builder removeChildFlowFile(final FlowFile childFlowFile) {
if (this.childrenUuids == null) {

View File

@ -0,0 +1,26 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-commons</artifactId>
<version>1.1.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-schema-utils</artifactId>
<dependencies>
</dependencies>
</project>

View File

@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class ComplexRecordField implements RecordField {
private static final FieldType fieldType = FieldType.COMPLEX;
private final String fieldName;
private final Repetition repetition;
private final List<RecordField> subFields;
public ComplexRecordField(final String fieldName, final Repetition repetition, final RecordField... subFields) {
this(fieldName, repetition, Stream.of(subFields).collect(Collectors.toList()));
}
public ComplexRecordField(final String fieldName, final Repetition repetition, final List<RecordField> subFields) {
Objects.requireNonNull(fieldName);
Objects.requireNonNull(repetition);
Objects.requireNonNull(subFields);
if (subFields.isEmpty()) {
throw new IllegalArgumentException("Cannot have a RecordField of type " + fieldType.name() + " without any sub-fields");
}
this.fieldName = fieldName;
this.repetition = repetition;
this.subFields = subFields;
}
@Override
public String getFieldName() {
return fieldName;
}
@Override
public FieldType getFieldType() {
return fieldType;
}
@Override
public Repetition getRepetition() {
return repetition;
}
@Override
public List<RecordField> getSubFields() {
return subFields;
}
@Override
public String toString() {
return "ComplexRecordField[" + fieldName + "]";
}
@Override
public int hashCode() {
return 81 + fieldName.hashCode();
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (!(obj instanceof RecordField)) {
return false;
}
final RecordField other = (RecordField) obj;
return fieldName.equals(other.getFieldName());
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.util.HashMap;
import java.util.Map;
public class FieldMapRecord implements Record {
private final Map<String, Object> values;
private final RecordSchema schema;
public FieldMapRecord(final Map<RecordField, Object> values, final RecordSchema schema) {
this.schema = schema;
this.values = convertFieldToName(values);
}
private static Map<String, Object> convertFieldToName(final Map<RecordField, Object> map) {
final Map<String, Object> nameMap = new HashMap<>(map.size());
for (final Map.Entry<RecordField, Object> entry : map.entrySet()) {
nameMap.put(entry.getKey().getFieldName(), entry.getValue());
}
return nameMap;
}
@Override
public Object getFieldValue(final RecordField field) {
return values.get(field.getFieldName());
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object getFieldValue(final String fieldName) {
return values.get(fieldName);
}
@Override
public String toString() {
return "FieldMapRecord[" + values + "]";
}
@Override
public int hashCode() {
return 33 + 41 * values.hashCode();
}
@Override
public boolean equals(final Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (!(obj instanceof FieldMapRecord)) {
return false;
}
final FieldMapRecord other = (FieldMapRecord) obj;
return values.equals(other.values);
}
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.util.Map;
public enum FieldType {
STRING(2, true, String.class), // 2 bytes for length of string
LONG_STRING(4, true, String.class), // 4 bytes for length of string
BOOLEAN(1, false, Boolean.class),
LONG(8, false, Long.class),
INT(4, false, Integer.class),
BYTE_ARRAY(4, true, byte[].class), // 4 bytes for number of bytes
COMPLEX(2, true, Record.class), // 2 bytes for number of sub-elements
MAP(2, true, Map.class),
UNION(4, false, NamedValue.class);
private final int numBytes;
private final boolean lengthEncoded;
private final Class<?> javaClass;
private FieldType(final int numBytes, final boolean lengthEncoded, final Class<?> javaClass) {
this.numBytes = numBytes;
this.lengthEncoded = lengthEncoded;
this.javaClass = javaClass;
}
/**
* Indicates the number of bytes that must be read for this field. If this field is length-encoded
* (see {@link #isLengthEncoded()}, then this method tells us how many bytes to read in order to determine
* the full length of the field. Otherwise, these bytes tell us the full length of the field themselves.
*
* @return the number of bytes to read for this field
*/
public int getNumberOfBytes() {
return numBytes;
}
/**
* This method returns <code>true</code>, then the value of {@link #getNumberOfBytes()} tells us how many bytes to read in
* order to determine the full length of the field (if the field is not complex) or the number of sub-fields to
* read in order to determine the full field (if this field is complex). If <code>false</code>, the value of
* {@link #getNumberOfBytes()} simply tells us how many bytes must be read in order to read the entire field.
*
* @return whether or not the field is length-encoded.
*/
public boolean isLengthEncoded() {
return lengthEncoded;
}
/**
* @return the Java type that corresponds to this FieldType
*/
public Class<?> getJavaClass() {
return javaClass;
}
}

View File

@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import static java.util.Objects.requireNonNull;
import java.util.ArrayList;
import java.util.List;
public class MapRecordField implements RecordField {
private final String fieldName;
private final RecordField keyField;
private final RecordField valueField;
private final Repetition repetition;
private final List<RecordField> subFields;
public MapRecordField(final String fieldName, final RecordField keyField, final RecordField valueField, final Repetition repetition) {
this.fieldName = requireNonNull(fieldName);
this.keyField = requireNonNull(keyField);
this.valueField = requireNonNull(valueField);
this.repetition = requireNonNull(repetition);
subFields = new ArrayList<>(2);
subFields.add(keyField);
subFields.add(valueField);
}
@Override
public String getFieldName() {
return fieldName;
}
@Override
public FieldType getFieldType() {
return FieldType.MAP;
}
@Override
public Repetition getRepetition() {
return repetition;
}
@Override
public List<RecordField> getSubFields() {
return subFields;
}
public RecordField getKeyField() {
return keyField;
}
public RecordField getValueField() {
return valueField;
}
@Override
public String toString() {
return "MapRecordField[" + fieldName + "]";
}
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
public class NamedValue {
private final String name;
private final Object value;
public NamedValue(final String name, final Object value) {
this.name = name;
this.value = value;
}
public String getName() {
return name;
}
public Object getValue() {
return value;
}
}

View File

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
public interface Record {
RecordSchema getSchema();
Object getFieldValue(String fieldName);
default Object getFieldValue(RecordField field) {
return getFieldValue(field.getFieldName());
}
}

View File

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.util.List;
public interface RecordField {
String getFieldName();
FieldType getFieldType();
Repetition getRepetition();
List<RecordField> getSubFields();
}

View File

@ -0,0 +1,188 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class RecordSchema {
private static final String FIELD_NAME = "Field Name";
private static final String FIELD_TYPE = "Field Type";
private static final String REPETITION = "Repetition";
private static final String SUBFIELDS = "SubFields";
private static final String STRING_TYPE = "String";
private static final String INT_TYPE = "Integer";
private static final String LONG_TYPE = "Long";
private static final String SUBFIELD_TYPE = "SubFieldList";
private final List<RecordField> fields;
public RecordSchema(final List<RecordField> fields) {
this.fields = fields;
}
public RecordSchema(final RecordField... fields) {
this(Arrays.asList(fields));
}
public List<RecordField> getFields() {
return fields;
}
public RecordField getField(final String fieldName) {
return fields.stream()
.filter(field -> field.getFieldName().equals(fieldName))
.findFirst()
.orElse(null);
}
public void writeTo(final OutputStream out) throws IOException {
try {
final DataOutputStream dos = (out instanceof DataOutputStream) ? (DataOutputStream) out : new DataOutputStream(out);
dos.writeInt(fields.size());
for (final RecordField field : fields) {
writeField(field, dos);
}
} catch (final IOException ioe) {
throw new IOException("Unable to write Record Schema to stream", ioe);
}
}
private void writeField(final RecordField field, final DataOutputStream dos) throws IOException {
dos.writeInt(4); // A field is made up of 4 "elements": Field Name, Field Type, Field Repetition, Sub-Fields.
// For each of the elements, we write a String indicating the Element Name, a String indicating the Element Type, and
// finally the Element data itself.
dos.writeUTF(FIELD_NAME);
dos.writeUTF(STRING_TYPE);
dos.writeUTF(field.getFieldName());
dos.writeUTF(FIELD_TYPE);
dos.writeUTF(STRING_TYPE);
dos.writeUTF(field.getFieldType().name());
dos.writeUTF(REPETITION);
dos.writeUTF(STRING_TYPE);
dos.writeUTF(field.getRepetition().name());
dos.writeUTF(SUBFIELDS);
dos.writeUTF(SUBFIELD_TYPE);
final List<RecordField> subFields = field.getSubFields();
dos.writeInt(subFields.size()); // SubField is encoded as number of Sub-Fields followed by the fields themselves.
for (final RecordField subField : subFields) {
writeField(subField, dos);
}
}
public static RecordSchema readFrom(final InputStream in) throws IOException {
try {
final DataInputStream dis = (in instanceof DataInputStream) ? (DataInputStream) in : new DataInputStream(in);
final int numFields = dis.readInt();
final List<RecordField> fields = new ArrayList<>(numFields);
for (int i = 0; i < numFields; i++) {
final RecordField field = readField(dis);
fields.add(field);
}
return new RecordSchema(fields);
} catch (final IOException ioe) {
throw new IOException("Unable to read Record Schema from stream", ioe);
}
}
@SuppressWarnings("unchecked")
private static RecordField readField(final DataInputStream dis) throws IOException {
final Map<String, Object> schemaFieldMap = new HashMap<>();
final int numElementsToRead = dis.readInt();
for (int i = 0; i < numElementsToRead; i++) {
final String fieldName = dis.readUTF();
final String typeName = dis.readUTF();
Object fieldValue = null;
switch (typeName) {
case STRING_TYPE:
fieldValue = dis.readUTF();
break;
case INT_TYPE:
fieldValue = dis.readInt();
break;
case LONG_TYPE:
fieldValue = dis.readLong();
break;
case SUBFIELD_TYPE: {
final int numFields = dis.readInt();
final List<RecordField> subFields = new ArrayList<>(numFields);
for (int j = 0; j < numFields; j++) {
subFields.add(readField(dis));
}
fieldValue = subFields;
break;
}
default: {
throw new IOException("Cannot read schema because the schema definition contains a field named '"
+ fieldName + "' with a Field Type of '" + typeName + "', which is not a known Field Type");
}
}
schemaFieldMap.put(fieldName, fieldValue);
}
final String fieldName = (String) schemaFieldMap.get(FIELD_NAME);
final String fieldTypeName = (String) schemaFieldMap.get(FIELD_TYPE);
final String repetitionName = (String) schemaFieldMap.get(REPETITION);
List<RecordField> subFields = (List<RecordField>) schemaFieldMap.get(SUBFIELDS);
if (subFields == null) {
subFields = Collections.emptyList();
}
final Repetition repetition = Repetition.valueOf(repetitionName);
if (FieldType.COMPLEX.name().equals(fieldTypeName)) {
return new ComplexRecordField(fieldName, repetition, subFields);
} else if (FieldType.UNION.name().equals(fieldTypeName)) {
return new UnionRecordField(fieldName, repetition, subFields);
} else if (FieldType.MAP.name().equals(fieldTypeName)) {
if (subFields.size() != 2) {
throw new IOException("Found a Map that did not have a 'Key' field and a 'Value' field but instead had " + subFields.size() + " fields: " + subFields);
}
final RecordField keyField = subFields.get(0);
final RecordField valueField = subFields.get(1);
return new MapRecordField(fieldName, keyField, valueField, repetition);
}
return new SimpleRecordField(fieldName, FieldType.valueOf(fieldTypeName), repetition);
}
@Override
public String toString() {
return "RecordSchema[" + fields + "]";
}
}

View File

@ -0,0 +1,22 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
public enum Repetition {
ZERO_OR_ONE, EXACTLY_ONE, ZERO_OR_MORE;
}

View File

@ -0,0 +1,191 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class SchemaRecordReader {
private final RecordSchema schema;
public SchemaRecordReader(final RecordSchema schema) {
this.schema = schema;
}
public static SchemaRecordReader fromSchema(final RecordSchema schema) {
return new SchemaRecordReader(schema);
}
private static void fillBuffer(final InputStream in, final byte[] destination) throws IOException {
int bytesRead = 0;
int len;
while (bytesRead < destination.length) {
len = in.read(destination, bytesRead, destination.length - bytesRead);
if (len < 0) {
throw new EOFException();
}
bytesRead += len;
}
}
public Record readRecord(final InputStream in) throws IOException {
final int sentinelByte = in.read();
if (sentinelByte < 0) {
return null;
}
if (sentinelByte != 1) {
throw new IOException("Expected to read a Sentinel Byte of '1' but got a value of '" + sentinelByte + "' instead");
}
final List<RecordField> schemaFields = schema.getFields();
final Map<RecordField, Object> fields = new HashMap<>(schemaFields.size());
for (final RecordField field : schema.getFields()) {
final Object value = readField(in, field);
fields.put(field, value);
}
return new FieldMapRecord(fields, schema);
}
private Object readField(final InputStream in, final RecordField field) throws IOException {
switch (field.getRepetition()) {
case ZERO_OR_MORE: {
// If repetition is 0+ then that means we have a list and need to read how many items are in the list.
final int iterations = readInt(in);
if (iterations == 0) {
return Collections.emptyList();
}
final List<Object> value = new ArrayList<>(iterations);
for (int i = 0; i < iterations; i++) {
value.add(readFieldValue(in, field.getFieldType(), field.getFieldName(), field.getSubFields()));
}
return value;
}
case ZERO_OR_ONE: {
// If repetition is 0 or 1 (optional), then check if next byte is a 0, which means field is absent or 1, which means
// field is present. Otherwise, throw an Exception.
final int nextByte = in.read();
if (nextByte == -1) {
throw new EOFException("Unexpected End-of-File when attempting to read Repetition value for field '" + field.getFieldName() + "'");
}
if (nextByte == 0) {
return null;
}
if (nextByte != 1) {
throw new IOException("Invalid Boolean value found when reading 'Repetition' of field '" + field.getFieldName() + "'. Expected 0 or 1 but got " + (nextByte & 0xFF));
}
}
}
return readFieldValue(in, field.getFieldType(), field.getFieldName(), field.getSubFields());
}
private Object readFieldValue(final InputStream in, final FieldType fieldType, final String fieldName, final List<RecordField> subFields) throws IOException {
switch (fieldType) {
case BOOLEAN: {
final DataInputStream dis = new DataInputStream(in);
return dis.readBoolean();
}
case INT: {
return readInt(in);
}
case LONG: {
final DataInputStream dis = new DataInputStream(in);
return dis.readLong();
}
case STRING: {
final DataInputStream dis = new DataInputStream(in);
return dis.readUTF();
}
case LONG_STRING: {
final int length = readInt(in);
final byte[] buffer = new byte[length];
fillBuffer(in, buffer);
return new String(buffer, StandardCharsets.UTF_8);
}
case BYTE_ARRAY: {
final int length = readInt(in);
final byte[] buffer = new byte[length];
fillBuffer(in, buffer);
return buffer;
}
case MAP: {
final int numEntries = readInt(in);
final RecordField keyField = subFields.get(0);
final RecordField valueField = subFields.get(1);
final Map<Object, Object> entries = new HashMap<>(numEntries);
for (int i = 0; i < numEntries; i++) {
final Object key = readField(in, keyField);
final Object value = readField(in, valueField);
entries.put(key, value);
}
return entries;
}
case COMPLEX: {
final int numSubFields = subFields.size();
final Map<RecordField, Object> subFieldValues = new HashMap<>(numSubFields);
for (int i = 0; i < numSubFields; i++) {
final Object subFieldValue = readField(in, subFields.get(i));
subFieldValues.put(subFields.get(i), subFieldValue);
}
return new FieldMapRecord(subFieldValues, new RecordSchema(subFields));
}
case UNION: {
final DataInputStream dis = new DataInputStream(in);
final String childFieldType = dis.readUTF();
final Optional<RecordField> fieldOption = subFields.stream().filter(field -> field.getFieldName().equals(childFieldType)).findFirst();
if (!fieldOption.isPresent()) {
throw new IOException("Found a field of type '" + childFieldType + "' but that was not in the expected list of types");
}
final RecordField matchingField = fieldOption.get();
return readField(in, matchingField);
}
default: {
throw new IOException("Unrecognized Field Type " + fieldType + " for field '" + fieldName + "'");
}
}
}
private int readInt(final InputStream in) throws IOException {
final byte[] buffer = new byte[4];
fillBuffer(in, buffer);
return ByteBuffer.wrap(buffer).getInt();
}
}

View File

@ -0,0 +1,139 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.List;
import java.util.Map;
public class SchemaRecordWriter {
public void writeRecord(final Record record, final OutputStream out) throws IOException {
// write sentinel value to indicate that there is a record. This allows the reader to then read one
// byte and check if -1. If so, the reader knows there are no more records. If not, then the reader
// knows that it should be able to continue reading.
out.write(1);
writeRecordFields(record, out);
}
private void writeRecordFields(final Record record, final OutputStream out) throws IOException {
final DataOutputStream dos = out instanceof DataOutputStream ? (DataOutputStream) out : new DataOutputStream(out);
for (final RecordField field : record.getSchema().getFields()) {
final Object value = record.getFieldValue(field);
try {
writeFieldRepetitionAndValue(field, value, dos);
} catch (final Exception e) {
throw new IOException("Failed to write field '" + field.getFieldName() + "'", e);
}
}
}
private void writeFieldRepetitionAndValue(final RecordField field, final Object value, final DataOutputStream dos) throws IOException {
switch (field.getRepetition()) {
case EXACTLY_ONE: {
if (value == null) {
throw new IllegalArgumentException("Record does not have a value for the '" + field.getFieldName() + "' but the field is required");
}
writeFieldValue(field, value, dos);
break;
}
case ZERO_OR_MORE: {
if (value == null) {
dos.writeInt(0);
break;
}
if (!(value instanceof Collection)) {
throw new IllegalArgumentException("Record contains a value of type '" + value.getClass() +
"' for the '" + field.getFieldName() + "' but expected a Collection because the Repetition for the field is " + field.getRepetition());
}
final Collection<?> collection = (Collection<?>) value;
dos.writeInt(collection.size());
for (final Object fieldValue : collection) {
writeFieldValue(field, fieldValue, dos);
}
break;
}
case ZERO_OR_ONE: {
if (value == null) {
dos.write(0);
break;
}
dos.write(1);
writeFieldValue(field, value, dos);
break;
}
}
}
@SuppressWarnings("unchecked")
private void writeFieldValue(final RecordField field, final Object value, final DataOutputStream out) throws IOException {
switch (field.getFieldType()) {
case BOOLEAN:
out.writeBoolean((boolean) value);
break;
case BYTE_ARRAY:
final byte[] array = (byte[]) value;
out.writeInt(array.length);
out.write(array);
break;
case INT:
out.writeInt((Integer) value);
break;
case LONG:
out.writeLong((Long) value);
break;
case STRING:
out.writeUTF((String) value);
break;
case LONG_STRING:
final byte[] charArray = ((String) value).getBytes(StandardCharsets.UTF_8);
out.writeInt(charArray.length);
out.write(charArray);
break;
case MAP:
final Map<Object, Object> map = (Map<Object, Object>) value;
out.writeInt(map.size());
final List<RecordField> subFields = field.getSubFields();
final RecordField keyField = subFields.get(0);
final RecordField valueField = subFields.get(1);
for (final Map.Entry<Object, Object> entry : map.entrySet()) {
writeFieldRepetitionAndValue(keyField, entry.getKey(), out);
writeFieldRepetitionAndValue(valueField, entry.getValue(), out);
}
break;
case UNION:
final NamedValue namedValue = (NamedValue) value;
out.writeUTF(namedValue.getName());
final Record childRecord = (Record) namedValue.getValue();
writeRecordFields(childRecord, out);
break;
case COMPLEX:
final Record record = (Record) value;
writeRecordFields(record, out);
break;
}
}
}

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public class SimpleRecordField implements RecordField {
private final String fieldName;
private final FieldType fieldType;
private final Repetition repetition;
public SimpleRecordField(final String fieldName, final FieldType fieldType, final Repetition repetition) {
Objects.requireNonNull(fieldName);
Objects.requireNonNull(fieldType);
Objects.requireNonNull(repetition);
this.fieldName = fieldName;
this.fieldType = fieldType;
this.repetition = repetition;
}
@Override
public String getFieldName() {
return fieldName;
}
@Override
public FieldType getFieldType() {
return fieldType;
}
@Override
public Repetition getRepetition() {
return repetition;
}
@Override
public List<RecordField> getSubFields() {
return Collections.emptyList();
}
@Override
public int hashCode() {
return 31 + fieldName.hashCode();
}
@Override
public boolean equals(final Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (!(obj instanceof RecordField)) {
return false;
}
final RecordField other = (RecordField) obj;
return fieldName.equals(other.getFieldName());
}
@Override
public String toString() {
return "SimpleRecordField[fieldName=" + fieldName + ", type=" + fieldType.name() + "]";
}
}

View File

@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import static java.util.Objects.requireNonNull;
import java.util.Arrays;
import java.util.List;
public class UnionRecordField implements RecordField {
private final String fieldName;
private final Repetition repetition;
private final List<RecordField> possibilities;
public UnionRecordField(final String fieldName, final Repetition repetition, final RecordField... possibilities) {
this(fieldName, repetition, Arrays.asList(possibilities));
}
public UnionRecordField(final String fieldName, final Repetition repetition, final List<RecordField> possibilities) {
this.fieldName = requireNonNull(fieldName);
this.repetition = requireNonNull(repetition);
this.possibilities = requireNonNull(possibilities);
}
@Override
public String getFieldName() {
return fieldName;
}
@Override
public FieldType getFieldType() {
return FieldType.UNION;
}
@Override
public Repetition getRepetition() {
return repetition;
}
@Override
public List<RecordField> getSubFields() {
return possibilities;
}
@Override
public String toString() {
return "UnionRecordField[name=" + fieldName + ", possible types=" + possibilities + "]";
}
}

View File

@ -0,0 +1,281 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.junit.Test;
public class TestSchemaRecordReader {
@Test
public void testReadExactlyOnceFields() throws IOException {
final List<RecordField> fields = new ArrayList<>();
fields.add(new SimpleRecordField("int", FieldType.INT, Repetition.EXACTLY_ONE));
fields.add(new SimpleRecordField("boolean", FieldType.BOOLEAN, Repetition.EXACTLY_ONE));
fields.add(new SimpleRecordField("byte array", FieldType.BYTE_ARRAY, Repetition.EXACTLY_ONE));
fields.add(new SimpleRecordField("long", FieldType.LONG, Repetition.EXACTLY_ONE));
fields.add(new SimpleRecordField("string", FieldType.STRING, Repetition.EXACTLY_ONE));
fields.add(new SimpleRecordField("long string", FieldType.LONG_STRING, Repetition.EXACTLY_ONE));
fields.add(new ComplexRecordField("complex", Repetition.EXACTLY_ONE,
new SimpleRecordField("key", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.EXACTLY_ONE)));
fields.add(new MapRecordField("map",
new SimpleRecordField("key", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.ZERO_OR_ONE), Repetition.EXACTLY_ONE));
fields.add(new UnionRecordField("union1", Repetition.EXACTLY_ONE, Arrays.asList(new RecordField[] {
new SimpleRecordField("one", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("two", FieldType.INT, Repetition.EXACTLY_ONE)
})));
fields.add(new UnionRecordField("union2", Repetition.EXACTLY_ONE, Arrays.asList(new RecordField[] {
new SimpleRecordField("one", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("two", FieldType.INT, Repetition.EXACTLY_ONE)
})));
final RecordSchema schema = new RecordSchema(fields);
final SchemaRecordReader reader = SchemaRecordReader.fromSchema(schema);
final byte[] buffer;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final DataOutputStream dos = new DataOutputStream(baos)) {
dos.write(1); // sentinel byte
dos.writeInt(42);
dos.writeBoolean(true);
final byte[] array = "hello".getBytes();
dos.writeInt(array.length);
dos.write(array);
dos.writeLong(42L);
dos.writeUTF("hello");
final String longString = "hello";
final byte[] longStringArray = longString.getBytes(StandardCharsets.UTF_8);
dos.writeInt(longStringArray.length);
dos.write(longStringArray);
dos.writeUTF("key");
dos.writeUTF("value");
dos.writeInt(2);
dos.writeUTF("key1");
dos.writeBoolean(true);
dos.writeUTF("value1");
dos.writeUTF("key2");
dos.writeBoolean(false);
dos.writeUTF("one");
dos.writeUTF("hello");
dos.writeUTF("two");
dos.writeInt(42);
buffer = baos.toByteArray();
}
try (final ByteArrayInputStream in = new ByteArrayInputStream(buffer)) {
final Record record = reader.readRecord(in);
assertNotNull(record);
assertEquals(42, record.getFieldValue("int"));
assertTrue((boolean) record.getFieldValue("boolean"));
assertTrue(Arrays.equals("hello".getBytes(), (byte[]) record.getFieldValue("byte array")));
assertEquals(42L, record.getFieldValue("long"));
assertEquals("hello", record.getFieldValue("string"));
assertEquals("hello", record.getFieldValue("long string"));
final Record complexRecord = (Record) record.getFieldValue("complex");
assertEquals("key", complexRecord.getFieldValue(new SimpleRecordField("key", FieldType.STRING, Repetition.EXACTLY_ONE)));
assertEquals("value", complexRecord.getFieldValue(new SimpleRecordField("value", FieldType.STRING, Repetition.EXACTLY_ONE)));
final Map<String, String> map = new HashMap<>();
map.put("key1", "value1");
map.put("key2", null);
assertEquals(map, record.getFieldValue("map"));
assertEquals("hello", record.getFieldValue("union1"));
assertEquals(42, record.getFieldValue("union2"));
}
}
@Test
@SuppressWarnings("unchecked")
public void testReadZeroOrOneFields() throws IOException {
final List<RecordField> fields = new ArrayList<>();
fields.add(new SimpleRecordField("int", FieldType.INT, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("int present", FieldType.INT, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("boolean", FieldType.BOOLEAN, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("boolean present", FieldType.BOOLEAN, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("byte array", FieldType.BYTE_ARRAY, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("byte array present", FieldType.BYTE_ARRAY, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long", FieldType.LONG, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long present", FieldType.LONG, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("string", FieldType.STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("string present", FieldType.STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long string", FieldType.LONG_STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long string present", FieldType.LONG_STRING, Repetition.ZERO_OR_ONE));
fields.add(new ComplexRecordField("complex", Repetition.ZERO_OR_ONE,
new SimpleRecordField("key", FieldType.STRING, Repetition.ZERO_OR_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.ZERO_OR_ONE)));
fields.add(new ComplexRecordField("complex present", Repetition.ZERO_OR_ONE,
new SimpleRecordField("key", FieldType.STRING, Repetition.ZERO_OR_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.ZERO_OR_ONE)));
fields.add(new MapRecordField("map",
new SimpleRecordField("key", FieldType.STRING, Repetition.ZERO_OR_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.ZERO_OR_MORE), Repetition.ZERO_OR_ONE));
fields.add(new MapRecordField("map present",
new SimpleRecordField("key", FieldType.STRING, Repetition.ZERO_OR_ONE),
new SimpleRecordField("value", FieldType.STRING, Repetition.ZERO_OR_MORE), Repetition.ZERO_OR_ONE));
fields.add(new UnionRecordField("union", Repetition.ZERO_OR_ONE, Arrays.asList(new RecordField[] {
new SimpleRecordField("one", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("two", FieldType.INT, Repetition.EXACTLY_ONE)
})));
fields.add(new UnionRecordField("union present", Repetition.ZERO_OR_ONE, Arrays.asList(new RecordField[] {
new SimpleRecordField("one", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("two", FieldType.INT, Repetition.ZERO_OR_MORE)
})));
final RecordSchema schema = new RecordSchema(fields);
final SchemaRecordReader reader = SchemaRecordReader.fromSchema(schema);
// for each field, make the first one missing and the second one present.
final byte[] buffer;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final DataOutputStream dos = new DataOutputStream(baos)) {
dos.write(1); // sentinel byte
dos.write(0);
dos.writeByte(1);
dos.writeInt(42);
dos.write(0);
dos.writeByte(1);
dos.writeBoolean(true);
final byte[] array = "hello".getBytes();
dos.write(0);
dos.writeByte(1);
dos.writeInt(array.length);
dos.write(array);
dos.write(0);
dos.writeByte(1);
dos.writeLong(42L);
dos.write(0);
dos.writeByte(1);
dos.writeUTF("hello");
final String longString = "hello";
final byte[] longStringArray = longString.getBytes(StandardCharsets.UTF_8);
dos.write(0);
dos.writeByte(1);
dos.writeInt(longStringArray.length);
dos.write(longStringArray);
dos.write(0);
dos.writeByte(1);
dos.writeByte(1);
dos.writeUTF("key");
dos.writeByte(0);
dos.writeBoolean(false); // map not present
dos.writeBoolean(true); // map present
dos.writeInt(2); // 2 entries in the map
dos.writeBoolean(true); // key present
dos.writeUTF("key1");
dos.writeInt(2); // 2 values
dos.writeUTF("one");
dos.writeUTF("two");
dos.writeBoolean(false); // key not present
dos.writeInt(1);
dos.writeUTF("three");
dos.writeBoolean(false);
dos.writeBoolean(true);
dos.writeUTF("two");
dos.writeInt(3); // 3 entries
dos.writeInt(1);
dos.writeInt(2);
dos.writeInt(3);
buffer = baos.toByteArray();
}
try (final ByteArrayInputStream in = new ByteArrayInputStream(buffer)) {
final Record record = reader.readRecord(in);
assertNotNull(record);
// Read everything into a map and make sure that no value is missing that has a name ending in " present"
final Map<String, Object> valueMap = new HashMap<>();
for (final RecordField field : record.getSchema().getFields()) {
final Object value = record.getFieldValue(field);
if (value == null) {
assertFalse(field.getFieldName().endsWith(" present"));
continue;
}
valueMap.put(field.getFieldName(), value);
}
assertEquals(42, valueMap.get("int present"));
assertTrue((boolean) valueMap.get("boolean present"));
assertTrue(Arrays.equals("hello".getBytes(), (byte[]) valueMap.get("byte array present")));
assertEquals(42L, valueMap.get("long present"));
assertEquals("hello", valueMap.get("string present"));
assertEquals("hello", valueMap.get("long string present"));
final Record complexRecord = (Record) valueMap.get("complex present");
assertEquals("key", complexRecord.getFieldValue(new SimpleRecordField("key", FieldType.STRING, Repetition.EXACTLY_ONE)));
assertNull(complexRecord.getFieldValue(new SimpleRecordField("value", FieldType.STRING, Repetition.EXACTLY_ONE)));
final Map<String, List<String>> map = (Map<String, List<String>>) valueMap.get("map present");
assertNotNull(map);
assertEquals(2, map.size());
assertTrue(map.containsKey(null));
assertTrue(map.containsKey("key1"));
final List<String> key1Values = Arrays.asList(new String[] {"one", "two"});
assertEquals(key1Values, map.get("key1"));
final List<String> nullKeyValues = Arrays.asList(new String[] {"three"});
assertEquals(nullKeyValues, map.get(null));
final List<Integer> unionValues = (List<Integer>) valueMap.get("union present");
assertEquals(3, unionValues.size());
assertEquals(1, unionValues.get(0).intValue());
assertEquals(2, unionValues.get(1).intValue());
assertEquals(3, unionValues.get(2).intValue());
}
}
}

View File

@ -0,0 +1,178 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.repository.schema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.junit.Test;
public class TestSchemaRecordReaderWriter {
@Test
@SuppressWarnings("unchecked")
public void testRoundTrip() throws IOException {
// Create a 'complex' record that contains two different types of fields - a string and an int.
final List<RecordField> complexFieldList1 = new ArrayList<>();
complexFieldList1.add(createField("string field", FieldType.STRING));
complexFieldList1.add(createField("int field", FieldType.INT));
final ComplexRecordField complexField1 = new ComplexRecordField("complex1", Repetition.EXACTLY_ONE, complexFieldList1);
final Map<RecordField, Object> complexMap1 = new LinkedHashMap<>();
final RecordField stringField = createField("string field", FieldType.STRING);
final RecordField intField = createField("int field", FieldType.INT);
complexMap1.put(stringField, "apples");
complexMap1.put(intField, 100);
final FieldMapRecord complexRecord1 = new FieldMapRecord(complexMap1, new RecordSchema(stringField, intField));
// Create another 'complex' record that contains two other types of fields - a long string and a long.
final List<RecordField> complexFieldList2 = new ArrayList<>();
complexFieldList2.add(createField("long string field", FieldType.LONG_STRING));
complexFieldList2.add(createField("long field", FieldType.LONG));
final ComplexRecordField complexField2 = new ComplexRecordField("complex2", Repetition.EXACTLY_ONE, complexFieldList2);
final Map<RecordField, Object> complexMap2 = new LinkedHashMap<>();
final RecordField longStringField = createField("long string field", FieldType.LONG_STRING);
final RecordField longField = createField("long field", FieldType.LONG);
complexMap2.put(longStringField, "oranges");
complexMap2.put(longField, Long.MAX_VALUE);
final FieldMapRecord complexRecord2 = new FieldMapRecord(complexMap2, new RecordSchema(longStringField, longField));
// Create a Union Field that indicates that the type could be either 'complex 1' or 'complex 2'
final UnionRecordField unionRecordField = new UnionRecordField("union", Repetition.ZERO_OR_MORE, Arrays.asList(new RecordField[] {complexField1, complexField2}));
// Create a Record Schema
final List<RecordField> fields = new ArrayList<>();
fields.add(new SimpleRecordField("int", FieldType.INT, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("int present", FieldType.INT, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("boolean", FieldType.BOOLEAN, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("boolean present", FieldType.BOOLEAN, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("byte array", FieldType.BYTE_ARRAY, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("byte array present", FieldType.BYTE_ARRAY, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long", FieldType.LONG, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long present", FieldType.LONG, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("string", FieldType.STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("string present", FieldType.STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long string", FieldType.LONG_STRING, Repetition.ZERO_OR_ONE));
fields.add(new SimpleRecordField("long string present", FieldType.LONG_STRING, Repetition.ZERO_OR_ONE));
fields.add(new ComplexRecordField("complex present", Repetition.EXACTLY_ONE,
new SimpleRecordField("color", FieldType.STRING, Repetition.ZERO_OR_ONE),
new SimpleRecordField("fruit", FieldType.STRING, Repetition.ZERO_OR_ONE)));
fields.add(new MapRecordField("map present",
new SimpleRecordField("key", FieldType.STRING, Repetition.EXACTLY_ONE),
new SimpleRecordField("value", FieldType.INT, Repetition.EXACTLY_ONE), Repetition.ZERO_OR_ONE));
fields.add(unionRecordField);
final RecordSchema schema = new RecordSchema(fields);
// Create a 'complex' record that contains two different elements.
final RecordField colorField = createField("color", FieldType.STRING);
final RecordField fruitField = createField("fruit", FieldType.STRING);
final Map<RecordField, Object> complexFieldMap = new LinkedHashMap<>();
complexFieldMap.put(colorField, "red");
complexFieldMap.put(fruitField, "apple");
// Create a simple map that can be used for a Map Field
final Map<String, Integer> simpleMap = new HashMap<>();
simpleMap.put("apples", 100);
// Create a Map of record fields to values, so that we can create a Record to write out
final Map<RecordField, Object> values = new LinkedHashMap<>();
values.put(createField("int", FieldType.INT), 42);
values.put(createField("int present", FieldType.INT), 42);
values.put(createField("boolean present", FieldType.BOOLEAN), true);
values.put(createField("byte array present", FieldType.BYTE_ARRAY), "Hello".getBytes());
values.put(createField("long present", FieldType.LONG), 42L);
values.put(createField("string present", FieldType.STRING), "Hello");
values.put(createField("long string present", FieldType.LONG_STRING), "Long Hello");
values.put(createField("complex present", FieldType.COMPLEX), new FieldMapRecord(complexFieldMap, new RecordSchema(colorField, fruitField)));
values.put(new MapRecordField("map present", createField("key", FieldType.STRING), createField("value", FieldType.INT), Repetition.EXACTLY_ONE), simpleMap);
values.put(unionRecordField, Arrays.asList(new NamedValue[] {
new NamedValue("complex1", complexRecord1),
new NamedValue("complex2", complexRecord2)}));
final FieldMapRecord originalRecord = new FieldMapRecord(values, schema);
// Write out a record and read it back in.
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
// Write the schema to the stream
schema.writeTo(baos);
// Write the record twice, to make sure that we're able to read/write multiple sequential records
final SchemaRecordWriter writer = new SchemaRecordWriter();
writer.writeRecord(originalRecord, baos);
writer.writeRecord(originalRecord, baos);
try (final InputStream in = new ByteArrayInputStream(baos.toByteArray())) {
// Read the Schema from the stream and create a Record Reader for reading records, based on this schema
final RecordSchema readSchema = RecordSchema.readFrom(in);
final SchemaRecordReader reader = SchemaRecordReader.fromSchema(readSchema);
// Read two records and verify the values.
for (int i=0; i < 2; i++) {
final Record record = reader.readRecord(in);
assertNotNull(record);
assertEquals(42, record.getFieldValue("int"));
assertEquals(42, record.getFieldValue("int present"));
assertEquals(true, record.getFieldValue("boolean present"));
assertTrue(Arrays.equals("Hello".getBytes(), (byte[]) record.getFieldValue("byte array present")));
assertEquals(42L, record.getFieldValue("long present"));
assertEquals("Hello", record.getFieldValue("string present"));
assertEquals("Long Hello", record.getFieldValue("long string present"));
final Record complexRecord = (Record) record.getFieldValue("complex present");
assertEquals("red", complexRecord.getFieldValue("color"));
assertEquals("apple", complexRecord.getFieldValue("fruit"));
assertEquals(simpleMap, record.getFieldValue("map present"));
final List<Record> unionRecords = (List<Record>) record.getFieldValue("union");
assertNotNull(unionRecords);
assertEquals(2, unionRecords.size());
final Record unionRecord1 = unionRecords.get(0);
assertEquals("apples", unionRecord1.getFieldValue("string field"));
assertEquals(100, unionRecord1.getFieldValue("int field"));
final Record unionRecord2 = unionRecords.get(1);
assertEquals("oranges", unionRecord2.getFieldValue("long string field"));
assertEquals(Long.MAX_VALUE, unionRecord2.getFieldValue("long field"));
}
// Ensure that there is no more data.
assertNull(reader.readRecord(in));
}
}
}
private SimpleRecordField createField(final String fieldName, final FieldType type) {
return new SimpleRecordField(fieldName, type, Repetition.ZERO_OR_ONE);
}
}

View File

@ -16,19 +16,445 @@
*/
package org.apache.nifi.stream.io;
import java.io.IOException;
import java.io.InputStream;
/**
* This class is a slight modification of the BufferedInputStream in the java.io package. The modification is that this implementation does not provide synchronization on method calls, which means
* that this class is not suitable for use by multiple threads. However, the absence of these synchronized blocks results in potentially much better performance.
*/
public class BufferedInputStream extends java.io.BufferedInputStream {
public class BufferedInputStream extends InputStream {
public BufferedInputStream(final InputStream in) {
super(in);
private final InputStream in;
private static int DEFAULT_BUFFER_SIZE = 8192;
/**
* The maximum size of array to allocate.
* Some VMs reserve some header words in an array.
* Attempts to allocate larger arrays may result in
* OutOfMemoryError: Requested array size exceeds VM limit
*/
private static int MAX_BUFFER_SIZE = Integer.MAX_VALUE - 8;
/**
* The internal buffer array where the data is stored. When necessary,
* it may be replaced by another array of
* a different size.
*/
protected byte buf[];
/**
* The index one greater than the index of the last valid byte in
* the buffer.
* This value is always
* in the range <code>0</code> through <code>buf.length</code>;
* elements <code>buf[0]</code> through <code>buf[count-1]
* </code>contain buffered input data obtained
* from the underlying input stream.
*/
private int count;
/**
* The current position in the buffer. This is the index of the next
* character to be read from the <code>buf</code> array.
* <p>
* This value is always in the range <code>0</code>
* through <code>count</code>. If it is less
* than <code>count</code>, then <code>buf[pos]</code>
* is the next byte to be supplied as input;
* if it is equal to <code>count</code>, then
* the next <code>read</code> or <code>skip</code>
* operation will require more bytes to be
* read from the contained input stream.
*
* @see java.io.BufferedInputStream#buf
*/
private int pos;
/**
* The value of the <code>pos</code> field at the time the last
* <code>mark</code> method was called.
* <p>
* This value is always
* in the range <code>-1</code> through <code>pos</code>.
* If there is no marked position in the input
* stream, this field is <code>-1</code>. If
* there is a marked position in the input
* stream, then <code>buf[markpos]</code>
* is the first byte to be supplied as input
* after a <code>reset</code> operation. If
* <code>markpos</code> is not <code>-1</code>,
* then all bytes from positions <code>buf[markpos]</code>
* through <code>buf[pos-1]</code> must remain
* in the buffer array (though they may be
* moved to another place in the buffer array,
* with suitable adjustments to the values
* of <code>count</code>, <code>pos</code>,
* and <code>markpos</code>); they may not
* be discarded unless and until the difference
* between <code>pos</code> and <code>markpos</code>
* exceeds <code>marklimit</code>.
*
* @see java.io.BufferedInputStream#mark(int)
* @see java.io.BufferedInputStream#pos
*/
protected int markpos = -1;
/**
* The maximum read ahead allowed after a call to the
* <code>mark</code> method before subsequent calls to the
* <code>reset</code> method fail.
* Whenever the difference between <code>pos</code>
* and <code>markpos</code> exceeds <code>marklimit</code>,
* then the mark may be dropped by setting
* <code>markpos</code> to <code>-1</code>.
*
* @see java.io.BufferedInputStream#mark(int)
* @see java.io.BufferedInputStream#reset()
*/
protected int marklimit;
/**
* Check to make sure that underlying input stream has not been
* nulled out due to close; if not return it;
*/
private InputStream getInIfOpen() throws IOException {
InputStream input = in;
if (input == null) {
throw new IOException("Stream closed");
}
return input;
}
public BufferedInputStream(final InputStream in, final int size) {
super(in, size);
/**
* Check to make sure that buffer has not been nulled out due to
* close; if not return it;
*/
private byte[] getBufIfOpen() throws IOException {
if (buf == null) {
throw new IOException("Stream closed");
}
return buf;
}
/**
* Creates a <code>BufferedInputStream</code>
* and saves its argument, the input stream
* <code>in</code>, for later use. An internal
* buffer array is created and stored in <code>buf</code>.
*
* @param in the underlying input stream.
*/
public BufferedInputStream(InputStream in) {
this(in, DEFAULT_BUFFER_SIZE);
}
/**
* Creates a <code>BufferedInputStream</code>
* with the specified buffer size,
* and saves its argument, the input stream
* <code>in</code>, for later use. An internal
* buffer array of length <code>size</code>
* is created and stored in <code>buf</code>.
*
* @param in the underlying input stream.
* @param size the buffer size.
* @exception IllegalArgumentException if {@code size <= 0}.
*/
public BufferedInputStream(InputStream in, int size) {
this.in = in;
if (size <= 0) {
throw new IllegalArgumentException("Buffer size <= 0");
}
buf = new byte[size];
}
/**
* Fills the buffer with more data, taking into account
* shuffling and other tricks for dealing with marks.
* Assumes that it is being called by a synchronized method.
* This method also assumes that all data has already been read in,
* hence pos > count.
*/
private void fill() throws IOException {
byte[] buffer = getBufIfOpen();
if (markpos < 0) {
pos = 0; /* no mark: throw away the buffer */
} else if (pos >= buffer.length) {
if (markpos > 0) { /* can throw away early part of the buffer */
int sz = pos - markpos;
System.arraycopy(buffer, markpos, buffer, 0, sz);
pos = sz;
markpos = 0;
} else if (buffer.length >= marklimit) {
markpos = -1; /* buffer got too big, invalidate mark */
pos = 0; /* drop buffer contents */
} else if (buffer.length >= MAX_BUFFER_SIZE) {
throw new OutOfMemoryError("Required array size too large");
} else { /* grow buffer */
int nsz = (pos <= MAX_BUFFER_SIZE - pos) ? pos * 2 : MAX_BUFFER_SIZE;
if (nsz > marklimit) {
nsz = marklimit;
}
byte nbuf[] = new byte[nsz];
System.arraycopy(buffer, 0, nbuf, 0, pos);
buffer = nbuf;
}
}
count = pos;
int n = getInIfOpen().read(buffer, pos, buffer.length - pos);
if (n > 0) {
count = n + pos;
}
}
/**
* See
* the general contract of the <code>read</code>
* method of <code>InputStream</code>.
*
* @return the next byte of data, or <code>-1</code> if the end of the
* stream is reached.
* @exception IOException if this input stream has been closed by
* invoking its {@link #close()} method,
* or an I/O error occurs.
* @see java.io.FilterInputStream#in
*/
@Override
public int read() throws IOException {
if (pos >= count) {
fill();
if (pos >= count) {
return -1;
}
}
return getBufIfOpen()[pos++] & 0xff;
}
/**
* Read characters into a portion of an array, reading from the underlying
* stream at most once if necessary.
*/
private int read1(byte[] b, int off, int len) throws IOException {
int avail = count - pos;
if (avail <= 0) {
/*
* If the requested length is at least as large as the buffer, and
* if there is no mark/reset activity, do not bother to copy the
* bytes into the local buffer. In this way buffered streams will
* cascade harmlessly.
*/
if (len >= getBufIfOpen().length && markpos < 0) {
return getInIfOpen().read(b, off, len);
}
fill();
avail = count - pos;
if (avail <= 0) {
return -1;
}
}
int cnt = (avail < len) ? avail : len;
System.arraycopy(getBufIfOpen(), pos, b, off, cnt);
pos += cnt;
return cnt;
}
/**
* Reads bytes from this byte-input stream into the specified byte array,
* starting at the given offset.
*
* <p>
* This method implements the general contract of the corresponding
* <code>{@link InputStream#read(byte[], int, int) read}</code> method of
* the <code>{@link InputStream}</code> class. As an additional
* convenience, it attempts to read as many bytes as possible by repeatedly
* invoking the <code>read</code> method of the underlying stream. This
* iterated <code>read</code> continues until one of the following
* conditions becomes true:
* <ul>
*
* <li>The specified number of bytes have been read,
*
* <li>The <code>read</code> method of the underlying stream returns
* <code>-1</code>, indicating end-of-file, or
*
* <li>The <code>available</code> method of the underlying stream
* returns zero, indicating that further input requests would block.
*
* </ul>
* If the first <code>read</code> on the underlying stream returns
* <code>-1</code> to indicate end-of-file then this method returns
* <code>-1</code>. Otherwise this method returns the number of bytes
* actually read.
*
* <p>
* Subclasses of this class are encouraged, but not required, to
* attempt to read as many bytes as possible in the same fashion.
*
* @param b destination buffer.
* @param off offset at which to start storing bytes.
* @param len maximum number of bytes to read.
* @return the number of bytes read, or <code>-1</code> if the end of
* the stream has been reached.
* @exception IOException if this input stream has been closed by
* invoking its {@link #close()} method,
* or an I/O error occurs.
*/
@Override
public int read(byte b[], int off, int len)
throws IOException {
getBufIfOpen(); // Check for closed stream
if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
throw new IndexOutOfBoundsException();
} else if (len == 0) {
return 0;
}
int n = 0;
for (;;) {
int nread = read1(b, off + n, len - n);
if (nread <= 0) {
return (n == 0) ? nread : n;
}
n += nread;
if (n >= len) {
return n;
}
// if not closed but no bytes available, return
InputStream input = in;
if (input != null && input.available() <= 0) {
return n;
}
}
}
/**
* See the general contract of the <code>skip</code>
* method of <code>InputStream</code>.
*
* @exception IOException if the stream does not support seek,
* or if this input stream has been closed by
* invoking its {@link #close()} method, or an
* I/O error occurs.
*/
@Override
public long skip(long n) throws IOException {
getBufIfOpen(); // Check for closed stream
if (n <= 0) {
return 0;
}
long avail = count - pos;
if (avail <= 0) {
// If no mark position set then don't keep in buffer
if (markpos < 0) {
return getInIfOpen().skip(n);
}
// Fill in buffer to save bytes for reset
fill();
avail = count - pos;
if (avail <= 0) {
return 0;
}
}
long skipped = (avail < n) ? avail : n;
pos += skipped;
return skipped;
}
/**
* Returns an estimate of the number of bytes that can be read (or
* skipped over) from this input stream without blocking by the next
* invocation of a method for this input stream. The next invocation might be
* the same thread or another thread. A single read or skip of this
* many bytes will not block, but may read or skip fewer bytes.
* <p>
* This method returns the sum of the number of bytes remaining to be read in
* the buffer (<code>count&nbsp;- pos</code>) and the result of calling the
* {@link java.io.FilterInputStream#in in}.available().
*
* @return an estimate of the number of bytes that can be read (or skipped
* over) from this input stream without blocking.
* @exception IOException if this input stream has been closed by
* invoking its {@link #close()} method,
* or an I/O error occurs.
*/
@Override
public int available() throws IOException {
int n = count - pos;
int avail = getInIfOpen().available();
return n > (Integer.MAX_VALUE - avail) ? Integer.MAX_VALUE : n + avail;
}
/**
* See the general contract of the <code>mark</code>
* method of <code>InputStream</code>.
*
* @param readlimit the maximum limit of bytes that can be read before
* the mark position becomes invalid.
* @see java.io.BufferedInputStream#reset()
*/
@Override
public void mark(int readlimit) {
marklimit = readlimit;
markpos = pos;
}
/**
* See the general contract of the <code>reset</code>
* method of <code>InputStream</code>.
* <p>
* If <code>markpos</code> is <code>-1</code>
* (no mark has been set or the mark has been
* invalidated), an <code>IOException</code>
* is thrown. Otherwise, <code>pos</code> is
* set equal to <code>markpos</code>.
*
* @exception IOException if this stream has not been marked or,
* if the mark has been invalidated, or the stream
* has been closed by invoking its {@link #close()}
* method, or an I/O error occurs.
* @see java.io.BufferedInputStream#mark(int)
*/
@Override
public void reset() throws IOException {
getBufIfOpen(); // Cause exception if closed
if (markpos < 0) {
throw new IOException("Resetting to invalid mark");
}
pos = markpos;
}
/**
* Tests if this input stream supports the <code>mark</code>
* and <code>reset</code> methods. The <code>markSupported</code>
* method of <code>BufferedInputStream</code> returns
* <code>true</code>.
*
* @return a <code>boolean</code> indicating if this stream type supports
* the <code>mark</code> and <code>reset</code> methods.
* @see java.io.InputStream#mark(int)
* @see java.io.InputStream#reset()
*/
@Override
public boolean markSupported() {
return true;
}
/**
* Closes this input stream and releases any system resources
* associated with the stream.
* Once the stream has been closed, further read(), available(), reset(),
* or skip() invocations will throw an IOException.
* Closing a previously closed stream has no effect.
*
* @exception IOException if an I/O error occurs.
*/
@Override
public void close() throws IOException {
this.in.close();
}
}

View File

@ -210,4 +210,42 @@ public class FormatUtils {
return sb.toString();
}
/**
* Formats nanoseconds in the format:
* 3 seconds, 8 millis, 3 nanos - if includeTotalNanos = false,
* 3 seconds, 8 millis, 3 nanos (3008000003 nanos) - if includeTotalNanos = true
*
* @param nanos the number of nanoseconds to format
* @param includeTotalNanos whether or not to include the total number of nanoseconds in parentheses in the returned value
* @return a human-readable String that is a formatted representation of the given number of nanoseconds.
*/
public static String formatNanos(final long nanos, final boolean includeTotalNanos) {
final StringBuilder sb = new StringBuilder();
final long seconds = nanos > 1000000000L ? nanos / 1000000000L : 0L;
long millis = nanos > 1000000L ? nanos / 1000000L : 0L;
final long nanosLeft = nanos % 1000000L;
if (seconds > 0) {
sb.append(seconds).append(" seconds");
}
if (millis > 0) {
if (seconds > 0) {
sb.append(", ");
millis -= seconds * 1000L;
}
sb.append(millis).append(" millis");
}
if (seconds > 0 || millis > 0) {
sb.append(", ");
}
sb.append(nanosLeft).append(" nanos");
if (includeTotalNanos) {
sb.append(" (").append(nanos).append(" nanos)");
}
return sb.toString();
}
}

View File

@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.util.timebuffer;
public class CountSizeEntityAccess implements EntityAccess<TimedCountSize> {
@Override
public TimedCountSize aggregate(final TimedCountSize oldValue, final TimedCountSize toAdd) {
if (oldValue == null && toAdd == null) {
return new TimedCountSize(0L, 0L);
} else if (oldValue == null) {
return toAdd;
} else if (toAdd == null) {
return oldValue;
}
return new TimedCountSize(oldValue.getCount() + toAdd.getCount(), oldValue.getSize() + toAdd.getSize());
}
@Override
public TimedCountSize createNew() {
return new TimedCountSize(0L, 0L);
}
@Override
public long getTimestamp(final TimedCountSize entity) {
return entity == null ? 0L : entity.getTimestamp();
}
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.util.timebuffer;
public class TimedCountSize {
private final long count;
private final long size;
private final long timestamp = System.currentTimeMillis();
public TimedCountSize(final long count, final long size) {
this.count = count;
this.size = size;
}
public long getCount() {
return count;
}
public long getSize() {
return size;
}
public long getTimestamp() {
return timestamp;
}
}

View File

@ -80,7 +80,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
private final Path partialPath;
private final Path snapshotPath;
private final SerDe<T> serde;
private final SerDeFactory<T> serdeFactory;
private final SyncListener syncListener;
private final FileChannel lockChannel;
private final AtomicLong transactionIdGenerator = new AtomicLong(0L);
@ -105,7 +105,15 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
private volatile boolean recovered = false;
public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDe<T> serde, final SyncListener syncListener) throws IOException {
this(new TreeSet<>(Collections.singleton(path)), partitionCount, serde, syncListener);
this(new TreeSet<>(Collections.singleton(path)), partitionCount, new SingletonSerDeFactory<T>(serde), syncListener);
}
public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDeFactory<T> serdeFactory, final SyncListener syncListener) throws IOException {
this(new TreeSet<>(Collections.singleton(path)), partitionCount, serdeFactory, syncListener);
}
public MinimalLockingWriteAheadLog(final SortedSet<Path> paths, final int partitionCount, final SerDe<T> serde, final SyncListener syncListener) throws IOException {
this(paths, partitionCount, new SingletonSerDeFactory<T>(serde), syncListener);
}
/**
@ -116,16 +124,16 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
* @param partitionCount the number of partitions/journals to use. For best
* performance, this should be close to the number of threads that are
* expected to update the repository simultaneously
* @param serde the serializer/deserializer for records
* @param serdeFactory the factory for the serializer/deserializer for records
* @param syncListener the listener
* @throws IOException if unable to initialize due to IO issue
*/
@SuppressWarnings("unchecked")
public MinimalLockingWriteAheadLog(final SortedSet<Path> paths, final int partitionCount, final SerDe<T> serde, final SyncListener syncListener) throws IOException {
public MinimalLockingWriteAheadLog(final SortedSet<Path> paths, final int partitionCount, final SerDeFactory<T> serdeFactory, final SyncListener syncListener) throws IOException {
this.syncListener = syncListener;
requireNonNull(paths);
requireNonNull(serde);
requireNonNull(serdeFactory);
if (paths.isEmpty()) {
throw new IllegalArgumentException("Paths must be non-empty");
@ -172,7 +180,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
this.basePath = paths.iterator().next();
this.partialPath = basePath.resolve("snapshot.partial");
this.snapshotPath = basePath.resolve("snapshot");
this.serde = serde;
this.serdeFactory = serdeFactory;
final Path lockPath = basePath.resolve("wali.lock");
lockChannel = new FileOutputStream(lockPath.toFile()).getChannel();
@ -189,7 +197,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
final Path partitionBasePath = pathIterator.next();
partitions[i] = new Partition<>(partitionBasePath.resolve("partition-" + i), serde, i, getVersion());
partitions[i] = new Partition<>(partitionBasePath.resolve("partition-" + i), serdeFactory, i, getVersion());
}
}
@ -242,13 +250,13 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
}
for (final T record : records) {
final UpdateType updateType = serde.getUpdateType(record);
final Object recordIdentifier = serde.getRecordIdentifier(record);
final UpdateType updateType = serdeFactory.getUpdateType(record);
final Object recordIdentifier = serdeFactory.getRecordIdentifier(record);
if (updateType == UpdateType.DELETE) {
recordMap.remove(recordIdentifier);
} else if (updateType == UpdateType.SWAP_OUT) {
final String newLocation = serde.getLocation(record);
final String newLocation = serdeFactory.getLocation(record);
if (newLocation == null) {
logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_OUT but "
+ "no indicator of where the Record is to be Swapped Out to; these records may be "
@ -258,7 +266,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
this.externalLocations.add(newLocation);
}
} else if (updateType == UpdateType.SWAP_IN) {
final String newLocation = serde.getLocation(record);
final String newLocation = serdeFactory.getLocation(record);
if (newLocation == null) {
logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_IN but no "
+ "indicator of where the Record is to be Swapped In from; these records may be duplicated "
@ -360,11 +368,14 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
+ waliImplementationVersion + " of the " + waliImplementationClass + " class; cannot restore using Version " + getVersion());
}
dataIn.readUTF(); // ignore serde class name for now
final String serdeEncoding = dataIn.readUTF(); // ignore serde class name for now
final int serdeVersion = dataIn.readInt();
final long maxTransactionId = dataIn.readLong();
final int numRecords = dataIn.readInt();
final SerDe<T> serde = serdeFactory.createSerDe(serdeEncoding);
serde.readHeader(dataIn);
for (int i = 0; i < numRecords; i++) {
final T record = serde.deserializeRecord(dataIn, serdeVersion);
if (record == null) {
@ -491,6 +502,8 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
long stopTheWorldNanos = -1L;
long stopTheWorldStart = -1L;
try {
final List<OutputStream> partitionStreams = new ArrayList<>();
writeLock.lock();
try {
stopTheWorldStart = System.nanoTime();
@ -512,25 +525,48 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
swapLocations = new HashSet<>(externalLocations);
for (final Partition<T> partition : partitions) {
try {
partition.rollover();
partitionStreams.add(partition.rollover());
} catch (final Throwable t) {
partition.blackList();
numberBlackListedPartitions.getAndIncrement();
throw t;
}
}
// notify global sync with the write lock held. We do this because we don't want the repository to get updated
// while the listener is performing its necessary tasks
if (syncListener != null) {
syncListener.onGlobalSync();
}
} finally {
writeLock.unlock();
}
stopTheWorldNanos = System.nanoTime() - stopTheWorldStart;
// Close all of the Partitions' Output Streams. We do this here, instead of in Partition.rollover()
// because we want to do this outside of the write lock. Because calling close() on FileOutputStream can
// be very expensive, as it has to flush the data to disk, we don't want to prevent other Process Sessions
// from getting committed. Since rollover() transitions the partition to write to a new file already, there
// is no reason that we need to close this FileOutputStream before releasing the write lock. Also, if any Exception
// does get thrown when calling close(), we don't need to blacklist the partition, as the stream that was getting
// closed is not the stream being written to for the partition anyway. We also catch any IOException and wait until
// after we've attempted to close all streams before we throw an Exception, to avoid resource leaks if one of them
// is unable to be closed (due to out of storage space, for instance).
IOException failure = null;
for (final OutputStream partitionStream : partitionStreams) {
try {
partitionStream.close();
} catch (final IOException e) {
failure = e;
}
}
if (failure != null) {
throw failure;
}
// notify global sync with the write lock held. We do this because we don't want the repository to get updated
// while the listener is performing its necessary tasks
if (syncListener != null) {
syncListener.onGlobalSync();
}
final SerDe<T> serde = serdeFactory.createSerDe(null);
// perform checkpoint, writing to .partial file
fileOut = new FileOutputStream(partialPath.toFile());
dataOut = new DataOutputStream(fileOut);
@ -540,6 +576,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
dataOut.writeInt(serde.getVersion());
dataOut.writeLong(maxTransactionId);
dataOut.writeInt(records.size());
serde.writeHeader(dataOut);
for (final T record : records) {
logger.trace("Checkpointing {}", record);
@ -627,7 +664,8 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
public static final String JOURNAL_EXTENSION = ".journal";
private static final Pattern JOURNAL_FILENAME_PATTERN = Pattern.compile("\\d+\\.journal");
private final SerDe<S> serde;
private final SerDeFactory<S> serdeFactory;
private SerDe<S> serde;
private final Path editDirectory;
private final int writeAheadLogVersion;
@ -650,9 +688,9 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
private final Queue<Path> recoveryFiles;
public Partition(final Path path, final SerDe<S> serde, final int partitionIndex, final int writeAheadLogVersion) throws IOException {
public Partition(final Path path, final SerDeFactory<S> serdeFactory, final int partitionIndex, final int writeAheadLogVersion) throws IOException {
this.editDirectory = path;
this.serde = serde;
this.serdeFactory = serdeFactory;
final File file = path.toFile();
if (!file.exists() && !file.mkdirs()) {
@ -744,24 +782,16 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
*
* @throws IOException if failure to rollover
*/
public void rollover() throws IOException {
public OutputStream rollover() throws IOException {
lock.lock();
try {
// Note that here we are closing fileOut and NOT dataOut. See the note in the close()
// method to understand the logic behind this.
final OutputStream out = fileOut;
if (out != null) {
try {
out.close();
} catch (final IOException ioe) {
dataOut = null;
fileOut = null;
blackList();
throw ioe;
}
}
final OutputStream oldOutputStream = fileOut;
dataOut = null;
fileOut = null;
this.serde = serdeFactory.createSerDe(null);
final Path editPath = getNewEditPath();
final FileOutputStream fos = new FileOutputStream(editPath.toFile());
try {
@ -770,10 +800,18 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
outStream.writeInt(writeAheadLogVersion);
outStream.writeUTF(serde.getClass().getName());
outStream.writeInt(serde.getVersion());
serde.writeHeader(outStream);
outStream.flush();
dataOut = outStream;
fileOut = fos;
} catch (final IOException ioe) {
try {
oldOutputStream.close();
} catch (final IOException ioe2) {
ioe.addSuppressed(ioe2);
}
logger.error("Failed to create new journal for {} due to {}", new Object[] {this, ioe.toString()}, ioe);
try {
fos.close();
@ -790,6 +828,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
currentJournalFilename = editPath.toFile().getName();
blackListed = false;
return oldOutputStream;
} finally {
lock.unlock();
}
@ -959,9 +998,11 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
+ "WALI version " + waliVersion + ", but the version used to restore it is only " + writeAheadLogVersion);
}
@SuppressWarnings("unused")
final String serdeClassName = recoveryIn.readUTF();
final String serdeEncoding = recoveryIn.readUTF();
this.recoveryVersion = recoveryIn.readInt();
serde = serdeFactory.createSerDe(serdeEncoding);
serde.readHeader(recoveryIn);
break;
}
@ -1009,12 +1050,15 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
final Path newEditPath = getNewEditPath();
this.serde = serdeFactory.createSerDe(null);
final FileOutputStream fos = new FileOutputStream(newEditPath.toFile());
final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos));
outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName());
outStream.writeInt(writeAheadLogVersion);
outStream.writeUTF(serde.getClass().getName());
outStream.writeInt(serde.getVersion());
serde.writeHeader(outStream);
outStream.flush();
dataOut = outStream;
fileOut = fos;

View File

@ -29,6 +29,15 @@ import java.util.Map;
*/
public interface SerDe<T> {
/**
* Provides the SerDe a chance to write header information to the given output stream
*
* @param out the DataOutputStream to write to
* @throws IOException if unable to write to the OutputStream
*/
default void writeHeader(DataOutputStream out) throws IOException {
}
/**
* <p>
* Serializes an Edit Record to the log via the given
@ -54,6 +63,15 @@ public interface SerDe<T> {
*/
void serializeRecord(T record, DataOutputStream out) throws IOException;
/**
* Provides the SerDe the opportunity to read header information before deserializing any records
*
* @param in the InputStream to read from
* @throws IOException if unable to read from the InputStream
*/
default void readHeader(DataInputStream in) throws IOException {
}
/**
* <p>
* Reads an Edit Record from the given {@link DataInputStream} and merges
@ -65,9 +83,9 @@ public interface SerDe<T> {
*
* @param in to deserialize from
* @param currentRecordStates an unmodifiable map of Record ID's to the
* current state of that record
* current state of that record
* @param version the version of the SerDe that was used to serialize the
* edit record
* edit record
* @return deserialized record
* @throws IOException if failure reading
*/
@ -125,4 +143,12 @@ public interface SerDe<T> {
* @return version
*/
int getVersion();
/**
* Closes any resources that the SerDe is holding open
*
* @throws IOException if unable to close resources
*/
default void close() throws IOException {
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.wali;
public interface SerDeFactory<T> {
/**
* Returns a new SerDe
*
* @param encodingName the name of encoding that was used when writing the serialized data, or <code>null</code> if
* the SerDe is to be used for serialization purposes
* @return a SerDe
*/
SerDe<T> createSerDe(String encodingName);
/**
* Returns the unique ID for the given record
*
* @param record to obtain identifier for
* @return identifier of record
*/
Object getRecordIdentifier(T record);
/**
* Returns the UpdateType for the given record
*
* @param record to retrieve update type for
* @return update type
*/
UpdateType getUpdateType(T record);
/**
* Returns the external location of the given record; this is used when a
* record is moved away from WALI or is being re-introduced to WALI. For
* example, WALI can be updated with a record of type
* {@link UpdateType#SWAP_OUT} that indicates a Location of
* file://tmp/external1 and can then be re-introduced to WALI by updating
* WALI with a record of type {@link UpdateType#CREATE} that indicates a
* Location of file://tmp/external1
*
* @param record to get location of
* @return location
*/
String getLocation(T record);
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.wali;
public class SingletonSerDeFactory<T> implements SerDeFactory<T> {
private final SerDe<T> serde;
public SingletonSerDeFactory(final SerDe<T> serde) {
this.serde = serde;
}
@Override
public SerDe<T> createSerDe(final String encodingName) {
return serde;
}
@Override
public Object getRecordIdentifier(final T record) {
return serde.getRecordIdentifier(record);
}
@Override
public UpdateType getUpdateType(final T record) {
return serde.getUpdateType(record);
}
@Override
public String getLocation(final T record) {
return serde.getLocation(record);
}
}

View File

@ -38,5 +38,6 @@
<module>nifi-site-to-site-client</module>
<module>nifi-hl7-query-language</module>
<module>nifi-hadoop-utils</module>
<module>nifi-schema-utils</module>
</modules>
</project>

View File

@ -64,4 +64,28 @@ public interface ResourceClaim extends Comparable<ResourceClaim> {
* @return <code>true</code> if the Resource Claim is in use, <code>false</code> otherwise
*/
boolean isInUse();
/**
* Provides the natural ordering for ResourceClaim objects. By default they are sorted by their id, then container, then section
*
* @param other other claim
* @return x such that x <= -1 if this is less than other;
* x=0 if this.equals(other);
* x >= 1 if this is greater than other
*/
@Override
default int compareTo(final ResourceClaim other) {
final int idComparison = getId().compareTo(other.getId());
if (idComparison != 0) {
return idComparison;
}
final int containerComparison = getContainer().compareTo(other.getContainer());
if (containerComparison != 0) {
return containerComparison;
}
return getSection().compareTo(other.getSection());
}
}

View File

@ -47,6 +47,10 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-schema-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>

View File

@ -17,6 +17,7 @@
package org.apache.nifi.controller;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
@ -28,34 +29,31 @@ import java.io.FilenameFilter;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.regex.Pattern;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.FlowFileSwapManager;
import org.apache.nifi.controller.repository.IncompleteSwapFileException;
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.SwapManagerInitializationContext;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.controller.swap.StandardSwapContents;
import org.apache.nifi.controller.swap.StandardSwapSummary;
import org.apache.nifi.controller.swap.SchemaSwapDeserializer;
import org.apache.nifi.controller.swap.SchemaSwapSerializer;
import org.apache.nifi.controller.swap.SimpleSwapDeserializer;
import org.apache.nifi.controller.swap.SwapDeserializer;
import org.apache.nifi.controller.swap.SwapSerializer;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -83,6 +81,8 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
private EventReporter eventReporter;
private ResourceClaimManager claimManager;
private static final byte[] MAGIC_HEADER = {'S', 'W', 'A', 'P'};
/**
* Default no args constructor for service loading only.
*/
@ -116,8 +116,14 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
final File swapTempFile = new File(swapFile.getParentFile(), swapFile.getName() + ".part");
final String swapLocation = swapFile.getAbsolutePath();
try (final FileOutputStream fos = new FileOutputStream(swapTempFile)) {
serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
final SwapSerializer serializer = new SchemaSwapSerializer();
try (final FileOutputStream fos = new FileOutputStream(swapTempFile);
final OutputStream out = new BufferedOutputStream(fos)) {
out.write(MAGIC_HEADER);
final DataOutputStream dos = new DataOutputStream(out);
dos.writeUTF(serializer.getSerializationName());
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, out);
fos.getFD().sync();
} catch (final IOException ioe) {
// we failed to write out the entire swap file. Delete the temporary file, if we can.
@ -154,14 +160,13 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
throw new FileNotFoundException("Failed to swap in FlowFiles from external storage location " + swapLocation + " into FlowFile Queue because the file could not be found");
}
final SwapContents swapContents;
try (final InputStream fis = new FileInputStream(swapFile);
final InputStream bis = new BufferedInputStream(fis);
final DataInputStream in = new DataInputStream(bis)) {
swapContents = deserializeFlowFiles(in, swapLocation, flowFileQueue, claimManager);
}
return swapContents;
final SwapDeserializer deserializer = createSwapDeserializer(in);
return deserializer.deserializeFlowFiles(in, swapLocation, flowFileQueue, claimManager);
}
}
@Override
@ -210,11 +215,13 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
// "<timestamp>-<queue identifier>-<random uuid>.swap". If we have two dashes, then we can just check if the queue ID is equal
// to the id of the queue given and if not we can just move on.
final String[] splits = swapFile.getName().split("-");
if (splits.length == 3) {
final String queueIdentifier = splits[1];
if (!queueIdentifier.equals(flowFileQueue.getIdentifier())) {
continue;
if (splits.length > 6) {
final String queueIdentifier = splits[1] + "-" + splits[2] + "-" + splits[3] + "-" + splits[4] + "-" + splits[5];
if (queueIdentifier.equals(flowFileQueue.getIdentifier())) {
swapLocations.add(swapFile.getAbsolutePath());
}
continue;
}
// Read the queue identifier from the swap file to check if the swap file is for this queue
@ -222,18 +229,22 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
final InputStream bufferedIn = new BufferedInputStream(fis);
final DataInputStream in = new DataInputStream(bufferedIn)) {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " because the encoding version is "
+ swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
final SwapDeserializer deserializer;
try {
deserializer = createSwapDeserializer(in);
} catch (final Exception e) {
final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " due to " + e;
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, errMsg);
throw new IOException(errMsg);
}
final String connectionId = in.readUTF();
if (connectionId.equals(flowFileQueue.getIdentifier())) {
swapLocations.add(swapFile.getAbsolutePath());
// If deserializer is not an instance of Simple Swap Deserializer, then it means that the serializer is new enough that
// we use the 3-element filename as illustrated above, so this is only necessary for the SimpleSwapDeserializer.
if (deserializer instanceof SimpleSwapDeserializer) {
final String connectionId = in.readUTF();
if (connectionId.equals(flowFileQueue.getIdentifier())) {
swapLocations.add(swapFile.getAbsolutePath());
}
}
}
}
@ -251,353 +262,36 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
final InputStream bufferedIn = new BufferedInputStream(fis);
final DataInputStream in = new DataInputStream(bufferedIn)) {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " because the encoding version is "
+ swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, errMsg);
throw new IOException(errMsg);
}
final int numRecords;
final long contentSize;
Long maxRecordId = null;
try {
in.readUTF(); // ignore Connection ID
numRecords = in.readInt();
contentSize = in.readLong();
if (numRecords == 0) {
return StandardSwapSummary.EMPTY_SUMMARY;
}
if (swapEncodingVersion > 7) {
maxRecordId = in.readLong();
}
} catch (final EOFException eof) {
logger.warn("Found premature End-of-File when reading Swap File {}. EOF occurred before any FlowFiles were encountered", swapLocation);
return StandardSwapSummary.EMPTY_SUMMARY;
}
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
final SwapContents swapContents = deserializeFlowFiles(in, queueSize, maxRecordId, swapEncodingVersion, true, claimManager, swapLocation);
return swapContents.getSummary();
final SwapDeserializer deserializer = createSwapDeserializer(in);
return deserializer.getSwapSummary(in, swapLocation, claimManager);
}
}
public static int serializeFlowFiles(final List<FlowFileRecord> toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream destination) throws IOException {
if (toSwap == null || toSwap.isEmpty()) {
return 0;
}
long contentSize = 0L;
for (final FlowFileRecord record : toSwap) {
contentSize += record.getSize();
}
private SwapDeserializer createSwapDeserializer(final DataInputStream dis) throws IOException {
dis.mark(MAGIC_HEADER.length);
// persist record to disk via the swap file
final OutputStream bufferedOut = new BufferedOutputStream(destination);
final DataOutputStream out = new DataOutputStream(bufferedOut);
final byte[] magicHeader = new byte[MAGIC_HEADER.length];
try {
out.writeInt(SWAP_ENCODING_VERSION);
out.writeUTF(queue.getIdentifier());
out.writeInt(toSwap.size());
out.writeLong(contentSize);
// get the max record id and write that out so that we know it quickly for restoration
long maxRecordId = 0L;
for (final FlowFileRecord flowFile : toSwap) {
if (flowFile.getId() > maxRecordId) {
maxRecordId = flowFile.getId();
}
}
out.writeLong(maxRecordId);
for (final FlowFileRecord flowFile : toSwap) {
out.writeLong(flowFile.getId());
out.writeLong(flowFile.getEntryDate());
out.writeLong(flowFile.getLineageStartDate());
out.writeLong(flowFile.getLineageStartIndex());
out.writeLong(flowFile.getLastQueueDate());
out.writeLong(flowFile.getQueueDateIndex());
out.writeLong(flowFile.getSize());
final ContentClaim claim = flowFile.getContentClaim();
if (claim == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
final ResourceClaim resourceClaim = claim.getResourceClaim();
out.writeUTF(resourceClaim.getId());
out.writeUTF(resourceClaim.getContainer());
out.writeUTF(resourceClaim.getSection());
out.writeLong(claim.getOffset());
out.writeLong(claim.getLength());
out.writeLong(flowFile.getContentClaimOffset());
out.writeBoolean(resourceClaim.isLossTolerant());
}
final Map<String, String> attributes = flowFile.getAttributes();
out.writeInt(attributes.size());
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
writeString(entry.getKey(), out);
writeString(entry.getValue(), out);
}
}
} finally {
out.flush();
}
logger.info("Successfully swapped out {} FlowFiles from {} to Swap File {}", toSwap.size(), queue, swapLocation);
return toSwap.size();
}
private static void writeString(final String toWrite, final OutputStream out) throws IOException {
final byte[] bytes = toWrite.getBytes(StandardCharsets.UTF_8);
final int utflen = bytes.length;
if (utflen < 65535) {
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
} else {
out.write(255);
out.write(255);
out.write(utflen >>> 24);
out.write(utflen >>> 16);
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
}
}
static SwapContents deserializeFlowFiles(final DataInputStream in, final String swapLocation, final FlowFileQueue queue, final ResourceClaimManager claimManager) throws IOException {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
throw new IOException("Cannot swap FlowFiles in from SwapFile because the encoding version is "
+ swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)");
}
final String connectionId = in.readUTF(); // Connection ID
if (!connectionId.equals(queue.getIdentifier())) {
throw new IllegalArgumentException("Cannot deserialize FlowFiles from Swap File at location " + swapLocation
+ " because those FlowFiles belong to Connection with ID " + connectionId + " and an attempt was made to swap them into a Connection with ID " + queue.getIdentifier());
}
int numRecords = 0;
long contentSize = 0L;
Long maxRecordId = null;
try {
numRecords = in.readInt();
contentSize = in.readLong(); // Content Size
if (swapEncodingVersion > 7) {
maxRecordId = in.readLong(); // Max Record ID
}
StreamUtils.fillBuffer(dis, magicHeader);
} catch (final EOFException eof) {
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections.emptyList());
final SwapContents partialContents = new StandardSwapContents(summary, Collections.emptyList());
throw new IncompleteSwapFileException(swapLocation, partialContents);
throw new IOException("Failed to read swap file because the file contained less than 4 bytes of data");
}
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
return deserializeFlowFiles(in, queueSize, maxRecordId, swapEncodingVersion, false, claimManager, swapLocation);
}
private static SwapContents deserializeFlowFiles(final DataInputStream in, final QueueSize queueSize, final Long maxRecordId,
final int serializationVersion, final boolean incrementContentClaims, final ResourceClaimManager claimManager, final String location) throws IOException {
final List<FlowFileRecord> flowFiles = new ArrayList<>(queueSize.getObjectCount());
final List<ResourceClaim> resourceClaims = new ArrayList<>(queueSize.getObjectCount());
Long maxId = maxRecordId;
for (int i = 0; i < queueSize.getObjectCount(); i++) {
try {
// legacy encoding had an "action" because it used to be couple with FlowFile Repository code
if (serializationVersion < 3) {
final int action = in.read();
if (action != 1) {
throw new IOException("Swap File is version " + serializationVersion + " but did not contain a 'UPDATE' record type");
}
}
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final long recordId = in.readLong();
if (maxId == null || recordId > maxId) {
maxId = recordId;
}
ffBuilder.id(recordId);
ffBuilder.entryDate(in.readLong());
if (serializationVersion > 1) {
// Lineage information was added in version 2
if (serializationVersion < 10) {
final int numLineageIdentifiers = in.readInt();
for (int lineageIdIdx = 0; lineageIdIdx < numLineageIdentifiers; lineageIdIdx++) {
in.readUTF(); //skip each identifier
}
}
// version 9 adds in a 'lineage start index'
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (serializationVersion > 8) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (serializationVersion > 5) {
// Version 9 adds in a 'queue date index'
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (serializationVersion > 8) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
ffBuilder.size(in.readLong());
if (serializationVersion < 3) {
readString(in); // connection Id
}
final boolean hasClaim = in.readBoolean();
ResourceClaim resourceClaim = null;
if (hasClaim) {
final String claimId;
if (serializationVersion < 5) {
claimId = String.valueOf(in.readLong());
} else {
claimId = in.readUTF();
}
final String container = in.readUTF();
final String section = in.readUTF();
final long resourceOffset;
final long resourceLength;
if (serializationVersion < 6) {
resourceOffset = 0L;
resourceLength = -1L;
} else {
resourceOffset = in.readLong();
resourceLength = in.readLong();
}
final long claimOffset = in.readLong();
final boolean lossTolerant;
if (serializationVersion >= 4) {
lossTolerant = in.readBoolean();
} else {
lossTolerant = false;
}
resourceClaim = claimManager.getResourceClaim(container, section, claimId);
if (resourceClaim == null) {
logger.error("Swap file indicates that FlowFile was referencing Resource Claim at container={}, section={}, claimId={}, "
+ "but this Resource Claim cannot be found! Will create a temporary Resource Claim, but this may affect the framework's "
+ "ability to properly clean up this resource", container, section, claimId);
resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, true);
}
final StandardContentClaim claim = new StandardContentClaim(resourceClaim, resourceOffset);
claim.setLength(resourceLength);
if (incrementContentClaims) {
claimManager.incrementClaimantCount(resourceClaim);
}
ffBuilder.contentClaim(claim);
ffBuilder.contentClaimOffset(claimOffset);
}
boolean attributesChanged = true;
if (serializationVersion < 3) {
attributesChanged = in.readBoolean();
}
if (attributesChanged) {
final int numAttributes = in.readInt();
for (int j = 0; j < numAttributes; j++) {
final String key = readString(in);
final String value = readString(in);
ffBuilder.addAttribute(key, value);
}
}
final FlowFileRecord record = ffBuilder.build();
if (resourceClaim != null) {
resourceClaims.add(resourceClaim);
}
flowFiles.add(record);
} catch (final EOFException eof) {
final SwapSummary swapSummary = new StandardSwapSummary(queueSize, maxId, resourceClaims);
final SwapContents partialContents = new StandardSwapContents(swapSummary, flowFiles);
throw new IncompleteSwapFileException(location, partialContents);
if (Arrays.equals(magicHeader, MAGIC_HEADER)) {
final String serializationName = dis.readUTF();
if (serializationName.equals(SchemaSwapDeserializer.getSerializationName())) {
return new SchemaSwapDeserializer();
}
}
final SwapSummary swapSummary = new StandardSwapSummary(queueSize, maxId, resourceClaims);
return new StandardSwapContents(swapSummary, flowFiles);
}
private static String readString(final InputStream in) throws IOException {
final Integer numBytes = readFieldLength(in);
if (numBytes == null) {
throw new EOFException();
}
final byte[] bytes = new byte[numBytes];
fillBuffer(in, bytes, numBytes);
return new String(bytes, StandardCharsets.UTF_8);
}
private static Integer readFieldLength(final InputStream in) throws IOException {
final int firstValue = in.read();
final int secondValue = in.read();
if (firstValue < 0) {
return null;
}
if (secondValue < 0) {
throw new EOFException();
}
if (firstValue == 0xff && secondValue == 0xff) {
final int ch1 = in.read();
final int ch2 = in.read();
final int ch3 = in.read();
final int ch4 = in.read();
if ((ch1 | ch2 | ch3 | ch4) < 0) {
throw new EOFException();
}
return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
throw new IOException("Cannot find a suitable Deserializer for swap file, written with Serialization Name '" + serializationName + "'");
} else {
return (firstValue << 8) + secondValue;
// SimpleSwapDeserializer is old and did not write out a magic header.
dis.reset();
return new SimpleSwapDeserializer();
}
}
private static void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
int bytesRead;
int totalBytesRead = 0;
while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
totalBytesRead += bytesRead;
}
if (totalBytesRead != length) {
throw new EOFException();
}
}
private void error(final String error) {
logger.error(error);

View File

@ -820,6 +820,7 @@ public class StandardFlowFileQueue implements FlowFileQueue {
long swapByteCount = 0L;
Long maxId = null;
List<ResourceClaim> resourceClaims = new ArrayList<>();
final long startNanos = System.nanoTime();
writeLock.lock();
try {
@ -866,6 +867,11 @@ public class StandardFlowFileQueue implements FlowFileQueue {
writeLock.unlock("Recover Swap Files");
}
if (!swapLocations.isEmpty()) {
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
logger.info("Recovered {} swap files for {} in {} millis", swapLocations.size(), this, millis);
}
return new StandardSwapSummary(new QueueSize(swapFlowFileCount, swapByteCount), maxId, resourceClaims);
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import java.util.Map;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.wali.SerDe;
import org.wali.UpdateType;
public abstract class RepositoryRecordSerde implements SerDe<RepositoryRecord> {
private Map<String, FlowFileQueue> flowFileQueueMap = null;
protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
this.flowFileQueueMap = queueMap;
}
protected Map<String, FlowFileQueue> getQueueMap() {
return flowFileQueueMap;
}
protected FlowFileQueue getFlowFileQueue(final String queueId) {
return flowFileQueueMap.get(queueId);
}
@Override
public Long getRecordIdentifier(final RepositoryRecord record) {
return record.getCurrent().getId();
}
@Override
public UpdateType getUpdateType(final RepositoryRecord record) {
switch (record.getType()) {
case CONTENTMISSING:
case DELETE:
return UpdateType.DELETE;
case CREATE:
return UpdateType.CREATE;
case UPDATE:
return UpdateType.UPDATE;
case SWAP_OUT:
return UpdateType.SWAP_OUT;
case SWAP_IN:
return UpdateType.SWAP_IN;
}
return null;
}
@Override
public String getLocation(final RepositoryRecord record) {
return record.getSwapLocation();
}
}

View File

@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import java.util.Map;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.wali.SerDe;
import org.wali.SerDeFactory;
import org.wali.UpdateType;
public class RepositoryRecordSerdeFactory implements SerDeFactory<RepositoryRecord> {
private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
private final ResourceClaimManager resourceClaimManager;
private Map<String, FlowFileQueue> flowFileQueueMap = null;
public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
this.resourceClaimManager = claimManager;
}
protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
this.flowFileQueueMap = queueMap;
}
protected Map<String, FlowFileQueue> getQueueMap() {
return flowFileQueueMap;
}
@Override
public SerDe<RepositoryRecord> createSerDe(final String encodingName) {
if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) {
final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager);
serde.setQueueMap(flowFileQueueMap);
return serde;
}
if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName)
|| LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) {
final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager);
serde.setQueueMap(flowFileQueueMap);
return serde;
}
throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known");
}
protected FlowFileQueue getFlowFileQueue(final String queueId) {
return flowFileQueueMap.get(queueId);
}
@Override
public Long getRecordIdentifier(final RepositoryRecord record) {
return record.getCurrent().getId();
}
@Override
public UpdateType getUpdateType(final RepositoryRecord record) {
switch (record.getType()) {
case CONTENTMISSING:
case DELETE:
return UpdateType.DELETE;
case CREATE:
return UpdateType.CREATE;
case UPDATE:
return UpdateType.UPDATE;
case SWAP_OUT:
return UpdateType.SWAP_OUT;
case SWAP_IN:
return UpdateType.SWAP_IN;
}
return null;
}
@Override
public String getLocation(final RepositoryRecord record) {
return record.getSwapLocation();
}
}

View File

@ -0,0 +1,213 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Map;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.schema.ContentClaimFieldMap;
import org.apache.nifi.controller.repository.schema.ContentClaimSchema;
import org.apache.nifi.controller.repository.schema.FlowFileSchema;
import org.apache.nifi.controller.repository.schema.RepositoryRecordFieldMap;
import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
import org.apache.nifi.controller.repository.schema.RepositoryRecordUpdate;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SchemaRecordReader;
import org.apache.nifi.repository.schema.SchemaRecordWriter;
import org.apache.nifi.repository.schema.SimpleRecordField;
import org.wali.SerDe;
import org.wali.UpdateType;
public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
private static final int MAX_ENCODING_VERSION = 1;
private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1;
private final RecordSchema contentClaimSchema = ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1;
private final ResourceClaimManager resourceClaimManager;
private volatile RecordSchema recoverySchema;
public SchemaRepositoryRecordSerde(final ResourceClaimManager resourceClaimManager) {
this.resourceClaimManager = resourceClaimManager;
}
@Override
public void writeHeader(final DataOutputStream out) throws IOException {
writeSchema.writeTo(out);
}
@Override
public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
serializeRecord(newRecordState, out);
}
@Override
public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
final RecordSchema schema;
switch (record.getType()) {
case CREATE:
case UPDATE:
schema = RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1;
break;
case CONTENTMISSING:
case DELETE:
schema = RepositoryRecordSchema.DELETE_SCHEMA_V1;
break;
case SWAP_IN:
schema = RepositoryRecordSchema.SWAP_IN_SCHEMA_V1;
break;
case SWAP_OUT:
schema = RepositoryRecordSchema.SWAP_OUT_SCHEMA_V1;
break;
default:
throw new IllegalArgumentException("Received Repository Record with unknown Update Type: " + record.getType()); // won't happen.
}
final RepositoryRecordFieldMap fieldMap = new RepositoryRecordFieldMap(record, schema, contentClaimSchema);
final RepositoryRecordUpdate update = new RepositoryRecordUpdate(fieldMap, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
new SchemaRecordWriter().writeRecord(update, out);
}
@Override
public void readHeader(final DataInputStream in) throws IOException {
recoverySchema = RecordSchema.readFrom(in);
}
@Override
public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
return deserializeRecord(in, version);
}
@Override
public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
final SchemaRecordReader reader = SchemaRecordReader.fromSchema(recoverySchema);
final Record updateRecord = reader.readRecord(in);
// Top level is always going to be a "Repository Record Update" record because we need a 'Union' type record at the
// top level that indicates which type of record we have.
final Record record = (Record) updateRecord.getFieldValue(RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V1);
final String actionType = (String) record.getFieldValue(RepositoryRecordSchema.ACTION_TYPE_FIELD);
final UpdateType updateType = UpdateType.valueOf(actionType);
switch (updateType) {
case CREATE:
return createRecord(record);
case DELETE:
return deleteRecord(record);
case SWAP_IN:
return swapInRecord(record);
case SWAP_OUT:
return swapOutRecord(record);
case UPDATE:
return updateRecord(record);
default:
throw new IOException("Found unrecognized Update Type '" + actionType + "'");
}
}
@SuppressWarnings("unchecked")
private StandardRepositoryRecord createRecord(final Record record) {
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
ffBuilder.id((Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID));
ffBuilder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
final Long lastQueueDate = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE);
final Long queueDateIndex = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX);
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
final Long lineageStartDate = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE);
final Long lineageStartIndex = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX);
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
populateContentClaim(ffBuilder, record);
ffBuilder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
ffBuilder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
final FlowFileRecord flowFileRecord = ffBuilder.build();
final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
final FlowFileQueue queue = getFlowFileQueue(queueId);
return new StandardRepositoryRecord(queue, flowFileRecord);
}
private void populateContentClaim(final StandardFlowFileRecord.Builder ffBuilder, final Record record) {
final Object claimMap = record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
if (claimMap == null) {
return;
}
final Record claimRecord = (Record) claimMap;
final ContentClaim contentClaim = ContentClaimFieldMap.getContentClaim(claimRecord, resourceClaimManager);
final Long offset = ContentClaimFieldMap.getContentClaimOffset(claimRecord);
ffBuilder.contentClaim(contentClaim);
ffBuilder.contentClaimOffset(offset);
}
private RepositoryRecord updateRecord(final Record record) {
return createRecord(record);
}
private RepositoryRecord deleteRecord(final Record record) {
final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
final FlowFileRecord flowFileRecord = ffBuilder.build();
final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
repoRecord.markForDelete();
return repoRecord;
}
private RepositoryRecord swapInRecord(final Record record) {
final StandardRepositoryRecord repoRecord = createRecord(record);
final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
repoRecord.setSwapLocation(swapLocation);
return repoRecord;
}
private RepositoryRecord swapOutRecord(final Record record) {
final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
final FlowFileQueue queue = getFlowFileQueue(queueId);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(recordId)
.build();
return new StandardRepositoryRecord(queue, flowFileRecord, swapLocation);
}
@Override
public int getVersion() {
return MAX_ENCODING_VERSION;
}
}

View File

@ -49,8 +49,6 @@ import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.io.ByteCountingInputStream;
import org.apache.nifi.controller.repository.io.ByteCountingOutputStream;
import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream;
import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream;
import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream;
@ -75,6 +73,8 @@ import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.ProvenanceReporter;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.ByteCountingInputStream;
import org.apache.nifi.stream.io.ByteCountingOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -121,8 +121,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
private int removedCount = 0; // number of flowfiles removed in this session
private long removedBytes = 0L; // size of all flowfiles removed in this session
private final AtomicLong bytesRead = new AtomicLong(0L);
private final AtomicLong bytesWritten = new AtomicLong(0L);
private long bytesRead = 0L;
private long bytesWritten = 0L;
private int flowFilesIn = 0, flowFilesOut = 0;
private long contentSizeIn = 0L, contentSizeOut = 0L;
@ -975,8 +975,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final Connectable connectable = context.getConnectable();
final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier());
flowFileEvent.setBytesRead(bytesRead.get());
flowFileEvent.setBytesWritten(bytesWritten.get());
flowFileEvent.setBytesRead(bytesRead);
flowFileEvent.setBytesWritten(bytesWritten);
// update event repository
try {
@ -1064,8 +1064,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
flowFilesOut = 0;
removedCount = 0;
removedBytes = 0L;
bytesRead.set(0L);
bytesWritten.set(0L);
bytesRead = 0L;
bytesWritten = 0L;
connectionCounts.clear();
createdFlowFiles.clear();
removedFlowFiles.clear();
@ -2006,8 +2006,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
// callback for reading FlowFile 1 and if we used the same stream we'd be destroying the ability to read from FlowFile 1.
if (allowCachingOfStream && recursionSet.isEmpty()) {
if (currentReadClaim == claim) {
if (currentReadClaimStream != null && currentReadClaimStream.getStreamLocation() <= offset) {
final long bytesToSkip = offset - currentReadClaimStream.getStreamLocation();
if (currentReadClaimStream != null && currentReadClaimStream.getBytesConsumed() <= offset) {
final long bytesToSkip = offset - currentReadClaimStream.getBytesConsumed();
if (bytesToSkip > 0) {
StreamUtils.skip(currentReadClaimStream, bytesToSkip);
}
@ -2023,7 +2023,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
}
currentReadClaim = claim;
currentReadClaimStream = new ByteCountingInputStream(rawInStream, new AtomicLong(0L));
currentReadClaimStream = new ByteCountingInputStream(rawInStream);
StreamUtils.skip(currentReadClaimStream, offset);
// Use a non-closeable stream because we want to keep it open after the callback has finished so that we can
@ -2270,8 +2270,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
writtenCount += footer.length;
}
} finally {
bytesWritten.getAndAdd(writtenCount);
bytesRead.getAndAdd(readCount);
bytesWritten += writtenCount;
bytesRead += readCount;
}
} catch (final ContentNotFoundException nfe) {
destroyContent(newClaim);
@ -2311,8 +2311,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
validateRecordState(source);
final StandardRepositoryRecord record = records.get(source);
long writtenToFlowFile = 0L;
ContentClaim newClaim = null;
final AtomicLong writtenHolder = new AtomicLong(0L);
try {
newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant());
claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source);
@ -2320,9 +2320,14 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
ensureNotAppending(newClaim);
try (final OutputStream stream = context.getContentRepository().write(newClaim);
final OutputStream disableOnClose = new DisableOnCloseOutputStream(stream);
final OutputStream countingOut = new ByteCountingOutputStream(disableOnClose, writtenHolder)) {
recursionSet.add(source);
writer.process(new FlowFileAccessOutputStream(countingOut, source));
final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnClose)) {
try {
recursionSet.add(source);
writer.process(new FlowFileAccessOutputStream(countingOut, source));
} finally {
writtenToFlowFile = countingOut.getBytesWritten();
bytesWritten += countingOut.getBytesWritten();
}
} finally {
recursionSet.remove(source);
}
@ -2342,8 +2347,6 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
resetWriteClaims(); // need to reset write claim before we can remove the claim
destroyContent(newClaim);
throw t;
} finally {
bytesWritten.getAndAdd(writtenHolder.get());
}
removeTemporaryClaim(record);
@ -2351,7 +2354,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
.fromFlowFile(record.getCurrent())
.contentClaim(newClaim)
.contentClaimOffset(0)
.size(writtenHolder.get())
.size(writtenToFlowFile)
.build();
record.setWorking(newFile);
@ -2379,7 +2382,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final OutputStream rawOutStream = context.getContentRepository().write(newClaim);
final OutputStream bufferedOutStream = new BufferedOutputStream(rawOutStream);
outStream = new ByteCountingOutputStream(bufferedOutStream, new AtomicLong(0L));
outStream = new ByteCountingOutputStream(bufferedOutStream);
originalByteWrittenCount = 0;
appendableStreams.put(newClaim, outStream);
@ -2448,7 +2451,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
} finally {
if (outStream != null) {
final long bytesWrittenThisIteration = outStream.getBytesWritten() - originalByteWrittenCount;
bytesWritten.getAndAdd(bytesWrittenThisIteration);
bytesWritten += bytesWrittenThisIteration;
}
}
@ -2542,8 +2545,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final StandardRepositoryRecord record = records.get(source);
final ContentClaim currClaim = record.getCurrentClaim();
long writtenToFlowFile = 0L;
ContentClaim newClaim = null;
final AtomicLong writtenHolder = new AtomicLong(0L);
try {
newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant());
claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source);
@ -2556,7 +2559,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final InputStream countingIn = new ByteCountingInputStream(disableOnCloseIn, bytesRead);
final OutputStream os = context.getContentRepository().write(newClaim);
final OutputStream disableOnCloseOut = new DisableOnCloseOutputStream(os);
final OutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut, writtenHolder)) {
final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut)) {
recursionSet.add(source);
@ -2574,6 +2577,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
cnfeThrown = true;
throw cnfe;
} finally {
writtenToFlowFile = countingOut.getBytesWritten();
recursionSet.remove(source);
// if cnfeThrown is true, we don't need to re-thrown the Exception; it will propagate.
@ -2595,7 +2599,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
destroyContent(newClaim);
throw t;
} finally {
bytesWritten.getAndAdd(writtenHolder.get());
bytesWritten += writtenToFlowFile;
}
removeTemporaryClaim(record);
@ -2603,7 +2607,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
.fromFlowFile(record.getCurrent())
.contentClaim(newClaim)
.contentClaimOffset(0L)
.size(writtenHolder.get())
.size(writtenToFlowFile)
.build();
record.setWorking(newFile);
@ -2635,8 +2639,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
long newSize = 0L;
try {
newSize = context.getContentRepository().importFrom(source, newClaim);
bytesWritten.getAndAdd(newSize);
bytesRead.getAndAdd(newSize);
bytesWritten += newSize;
bytesRead += newSize;
} catch (final Throwable t) {
destroyContent(newClaim);
throw new FlowFileAccessException("Failed to import data from " + source + " for " + destination + " due to " + t.toString(), t);
@ -2671,7 +2675,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
claimLog.debug("Creating ContentClaim {} for 'importFrom' for {}", newClaim, destination);
newSize = context.getContentRepository().importFrom(source, newClaim);
bytesWritten.getAndAdd(newSize);
bytesWritten += newSize;
} catch (final IOException e) {
throw new FlowFileAccessException("Unable to create ContentClaim due to " + e.toString(), e);
}
@ -2697,8 +2701,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
ensureNotAppending(record.getCurrentClaim());
final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize());
bytesRead.getAndAdd(copyCount);
bytesWritten.getAndAdd(copyCount);
bytesRead += copyCount;
bytesWritten += copyCount;
} catch (final ContentNotFoundException nfe) {
handleContentNotFound(nfe, record);
} catch (final Throwable t) {
@ -3016,8 +3020,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
this.removedCount += session.removedCount;
this.removedBytes += session.removedBytes;
this.bytesRead += session.bytesRead.get();
this.bytesWritten += session.bytesWritten.get();
this.bytesRead += session.bytesRead;
this.bytesWritten += session.bytesWritten;
this.flowFilesIn += session.flowFilesIn;
this.flowFilesOut += session.flowFilesOut;
this.contentSizeIn += session.contentSizeIn;

View File

@ -16,12 +16,7 @@
*/
package org.apache.nifi.controller.repository;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
@ -45,16 +40,12 @@ import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.wali.MinimalLockingWriteAheadLog;
import org.wali.SerDe;
import org.wali.SyncListener;
import org.wali.UpdateType;
import org.wali.WriteAheadRepository;
/**
@ -95,7 +86,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
// effectively final
private WriteAheadRepository<RepositoryRecord> wal;
private WriteAheadRecordSerde serde;
private RepositoryRecordSerdeFactory serdeFactory;
private ResourceClaimManager claimManager;
// WALI Provides the ability to register callbacks for when a Partition or the entire Repository is sync'ed with the underlying disk.
@ -153,8 +144,8 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
// TODO: Allow for backup path that can be used if disk out of space?? Would allow a snapshot to be stored on
// backup and then the data deleted from the normal location; then can move backup to normal location and
// delete backup. On restore, if no files exist in partition's directory, would have to check backup directory
serde = new WriteAheadRecordSerde(claimManager);
wal = new MinimalLockingWriteAheadLog<>(flowFileRepositoryPath, numPartitions, serde, this);
serdeFactory = new RepositoryRecordSerdeFactory(claimManager);
wal = new MinimalLockingWriteAheadLog<>(flowFileRepositoryPath, numPartitions, serdeFactory, this);
}
@Override
@ -319,6 +310,10 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
repoRecords.add(repoRecord);
}
// TODO: We should probably update this to support bulk 'SWAP OUT' records. As-is, we have to write out a
// 'SWAP OUT' record for each FlowFile, which includes the Update Type, FlowFile ID, swap file location, and Queue ID.
// We could instead have a single record with Update Type of 'SWAP OUT' and just include swap file location, Queue ID,
// and all FlowFile ID's.
// update WALI to indicate that the records were swapped out.
wal.update(repoRecords, true);
@ -347,9 +342,9 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
for (final FlowFileQueue queue : queueProvider.getAllQueues()) {
queueMap.put(queue.getIdentifier(), queue);
}
serde.setQueueMap(queueMap);
serdeFactory.setQueueMap(queueMap);
final Collection<RepositoryRecord> recordList = wal.recoverRecords();
serde.setQueueMap(null);
serdeFactory.setQueueMap(null);
for (final RepositoryRecord record : recordList) {
final ContentClaim claim = record.getCurrentClaim();
@ -361,7 +356,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
// Determine the next sequence number for FlowFiles
long maxId = minimumSequenceNumber;
for (final RepositoryRecord record : recordList) {
final long recordId = serde.getRecordIdentifier(record);
final long recordId = serdeFactory.getRecordIdentifier(record);
if (recordId > maxId) {
maxId = recordId;
}
@ -414,526 +409,4 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
public int checkpoint() throws IOException {
return wal.checkpoint();
}
private static class WriteAheadRecordSerde implements SerDe<RepositoryRecord> {
private static final int CURRENT_ENCODING_VERSION = 9;
public static final byte ACTION_CREATE = 0;
public static final byte ACTION_UPDATE = 1;
public static final byte ACTION_DELETE = 2;
public static final byte ACTION_SWAPPED_OUT = 3;
public static final byte ACTION_SWAPPED_IN = 4;
private Map<String, FlowFileQueue> flowFileQueueMap = null;
private long recordsRestored = 0L;
private final ResourceClaimManager claimManager;
public WriteAheadRecordSerde(final ResourceClaimManager claimManager) {
this.claimManager = claimManager;
}
private void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
this.flowFileQueueMap = queueMap;
}
@Override
public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out) throws IOException {
serializeEdit(previousRecordState, record, out, false);
}
public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten) throws IOException {
if (record.isMarkedForAbort()) {
logger.warn("Repository Record {} is marked to be aborted; it will be persisted in the FlowFileRepository as a DELETE record", record);
out.write(ACTION_DELETE);
out.writeLong(getRecordIdentifier(record));
serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
return;
}
final UpdateType updateType = getUpdateType(record);
if (updateType.equals(UpdateType.DELETE)) {
out.write(ACTION_DELETE);
out.writeLong(getRecordIdentifier(record));
serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
return;
}
// If there's a Destination Connection, that's the one that we want to associated with this record.
// However, on restart, we will restore the FlowFile and set this connection to its "originalConnection".
// If we then serialize the FlowFile again before it's transferred, it's important to allow this to happen,
// so we use the originalConnection instead
FlowFileQueue associatedQueue = record.getDestination();
if (associatedQueue == null) {
associatedQueue = record.getOriginalQueue();
}
if (updateType.equals(UpdateType.SWAP_OUT)) {
out.write(ACTION_SWAPPED_OUT);
out.writeLong(getRecordIdentifier(record));
out.writeUTF(associatedQueue.getIdentifier());
out.writeUTF(getLocation(record));
return;
}
final FlowFile flowFile = record.getCurrent();
final ContentClaim claim = record.getCurrentClaim();
switch (updateType) {
case UPDATE:
out.write(ACTION_UPDATE);
break;
case CREATE:
out.write(ACTION_CREATE);
break;
case SWAP_IN:
out.write(ACTION_SWAPPED_IN);
break;
default:
throw new AssertionError();
}
out.writeLong(getRecordIdentifier(record));
out.writeLong(flowFile.getEntryDate());
out.writeLong(flowFile.getLineageStartDate());
out.writeLong(flowFile.getLineageStartIndex());
final Long queueDate = flowFile.getLastQueueDate();
out.writeLong(queueDate == null ? System.currentTimeMillis() : queueDate);
out.writeLong(flowFile.getQueueDateIndex());
out.writeLong(flowFile.getSize());
if (associatedQueue == null) {
logger.warn("{} Repository Record {} has no Connection associated with it; it will be destroyed on restart",
new Object[]{this, record});
writeString("", out);
} else {
writeString(associatedQueue.getIdentifier(), out);
}
serializeContentClaim(claim, record.getCurrentClaimOffset(), out);
if (forceAttributesWritten || record.isAttributesChanged() || updateType == UpdateType.CREATE || updateType == UpdateType.SWAP_IN) {
out.write(1); // indicate attributes changed
final Map<String, String> attributes = flowFile.getAttributes();
out.writeInt(attributes.size());
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
writeString(entry.getKey(), out);
writeString(entry.getValue(), out);
}
} else {
out.write(0); // indicate attributes did not change
}
if (updateType == UpdateType.SWAP_IN) {
out.writeUTF(record.getSwapLocation());
}
}
@Override
public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
final int action = in.read();
final long recordId = in.readLong();
if (action == ACTION_DELETE) {
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
if (version > 4) {
deserializeClaim(in, version, ffBuilder);
}
final FlowFileRecord flowFileRecord = ffBuilder.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
record.markForDelete();
return record;
}
if (action == ACTION_SWAPPED_OUT) {
final String queueId = in.readUTF();
final String location = in.readUTF();
final FlowFileQueue queue = flowFileQueueMap.get(queueId);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(recordId)
.build();
return new StandardRepositoryRecord(queue, flowFileRecord, location);
}
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final RepositoryRecord record = currentRecordStates.get(recordId);
ffBuilder.id(recordId);
if (record != null) {
ffBuilder.fromFlowFile(record.getCurrent());
}
ffBuilder.entryDate(in.readLong());
if (version > 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
}
}
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (version > 7) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (version > 5) {
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (version > 7) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
ffBuilder.size(in.readLong());
final String connectionId = readString(in);
logger.debug("{} -> {}", new Object[]{recordId, connectionId});
deserializeClaim(in, version, ffBuilder);
// recover new attributes, if they changed
final int attributesChanged = in.read();
if (attributesChanged == -1) {
throw new EOFException();
} else if (attributesChanged == 1) {
final int numAttributes = in.readInt();
final Map<String, String> attributes = new HashMap<>();
for (int i = 0; i < numAttributes; i++) {
final String key = readString(in);
final String value = readString(in);
attributes.put(key, value);
}
ffBuilder.addAttributes(attributes);
} else if (attributesChanged != 0) {
throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+ attributesChanged + " after successfully restoring " + recordsRestored + " records. The FlowFile Repository appears to be corrupt!");
}
final FlowFileRecord flowFile = ffBuilder.build();
String swapLocation = null;
if (action == ACTION_SWAPPED_IN) {
swapLocation = in.readUTF();
}
final StandardRepositoryRecord standardRepoRecord;
if (flowFileQueueMap == null) {
standardRepoRecord = new StandardRepositoryRecord(null, flowFile);
} else {
final FlowFileQueue queue = flowFileQueueMap.get(connectionId);
standardRepoRecord = new StandardRepositoryRecord(queue, flowFile);
if (swapLocation != null) {
standardRepoRecord.setSwapLocation(swapLocation);
}
if (connectionId.isEmpty()) {
logger.warn("{} does not have a Queue associated with it; this record will be discarded", flowFile);
standardRepoRecord.markForAbort();
} else if (queue == null) {
logger.warn("{} maps to unknown Queue {}; this record will be discarded", flowFile, connectionId);
standardRepoRecord.markForAbort();
}
}
recordsRestored++;
return standardRepoRecord;
}
@Override
public StandardRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
final int action = in.read();
if (action == -1) {
return null;
}
final long recordId = in.readLong();
if (action == ACTION_DELETE) {
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
if (version > 4) {
deserializeClaim(in, version, ffBuilder);
}
final FlowFileRecord flowFileRecord = ffBuilder.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
record.markForDelete();
return record;
}
// if action was not delete, it must be create/swap in
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final long entryDate = in.readLong();
if (version > 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
}
}
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (version > 7) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (version > 5) {
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (version > 7) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
final long size = in.readLong();
final String connectionId = readString(in);
logger.debug("{} -> {}", new Object[]{recordId, connectionId});
ffBuilder.id(recordId);
ffBuilder.entryDate(entryDate);
ffBuilder.size(size);
deserializeClaim(in, version, ffBuilder);
final int attributesChanged = in.read();
if (attributesChanged == 1) {
final int numAttributes = in.readInt();
final Map<String, String> attributes = new HashMap<>();
for (int i = 0; i < numAttributes; i++) {
final String key = readString(in);
final String value = readString(in);
attributes.put(key, value);
}
ffBuilder.addAttributes(attributes);
} else if (attributesChanged == -1) {
throw new EOFException();
} else if (attributesChanged != 0) {
throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+ attributesChanged + " after successfully restoring " + recordsRestored + " records");
}
final FlowFileRecord flowFile = ffBuilder.build();
String swapLocation = null;
if (action == ACTION_SWAPPED_IN) {
swapLocation = in.readUTF();
}
final StandardRepositoryRecord record;
if (flowFileQueueMap == null) {
record = new StandardRepositoryRecord(null, flowFile);
} else {
final FlowFileQueue queue = flowFileQueueMap.get(connectionId);
record = new StandardRepositoryRecord(queue, flowFile);
if (swapLocation != null) {
record.setSwapLocation(swapLocation);
}
if (connectionId.isEmpty()) {
logger.warn("{} does not have a FlowFile Queue associated with it; this record will be discarded", flowFile);
record.markForAbort();
} else if (queue == null) {
logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId);
record.markForAbort();
}
}
recordsRestored++;
return record;
}
@Override
public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
serializeEdit(null, record, out, true);
}
private void serializeContentClaim(final ContentClaim claim, final long offset, final DataOutputStream out) throws IOException {
if (claim == null) {
out.write(0);
} else {
out.write(1);
final ResourceClaim resourceClaim = claim.getResourceClaim();
writeString(resourceClaim.getId(), out);
writeString(resourceClaim.getContainer(), out);
writeString(resourceClaim.getSection(), out);
out.writeLong(claim.getOffset());
out.writeLong(claim.getLength());
out.writeLong(offset);
out.writeBoolean(resourceClaim.isLossTolerant());
}
}
private void deserializeClaim(final DataInputStream in, final int serializationVersion, final StandardFlowFileRecord.Builder ffBuilder) throws IOException {
// determine current Content Claim.
final int claimExists = in.read();
if (claimExists == 1) {
final String claimId;
if (serializationVersion < 4) {
claimId = String.valueOf(in.readLong());
} else {
claimId = readString(in);
}
final String container = readString(in);
final String section = readString(in);
final long resourceOffset;
final long resourceLength;
if (serializationVersion < 7) {
resourceOffset = 0L;
resourceLength = -1L;
} else {
resourceOffset = in.readLong();
resourceLength = in.readLong();
}
final long claimOffset = in.readLong();
final boolean lossTolerant;
if (serializationVersion >= 3) {
lossTolerant = in.readBoolean();
} else {
lossTolerant = false;
}
final ResourceClaim resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, false);
final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
contentClaim.setLength(resourceLength);
ffBuilder.contentClaim(contentClaim);
ffBuilder.contentClaimOffset(claimOffset);
} else if (claimExists == -1) {
throw new EOFException();
} else if (claimExists != 0) {
throw new IOException("Claim Existence Qualifier not found in stream; found value: "
+ claimExists + " after successfully restoring " + recordsRestored + " records");
}
}
private void writeString(final String toWrite, final OutputStream out) throws IOException {
final byte[] bytes = toWrite.getBytes("UTF-8");
final int utflen = bytes.length;
if (utflen < 65535) {
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
} else {
out.write(255);
out.write(255);
out.write(utflen >>> 24);
out.write(utflen >>> 16);
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
}
}
private String readString(final InputStream in) throws IOException {
final Integer numBytes = readFieldLength(in);
if (numBytes == null) {
throw new EOFException();
}
final byte[] bytes = new byte[numBytes];
fillBuffer(in, bytes, numBytes);
return new String(bytes, "UTF-8");
}
private Integer readFieldLength(final InputStream in) throws IOException {
final int firstValue = in.read();
final int secondValue = in.read();
if (firstValue < 0) {
return null;
}
if (secondValue < 0) {
throw new EOFException();
}
if (firstValue == 0xff && secondValue == 0xff) {
final int ch1 = in.read();
final int ch2 = in.read();
final int ch3 = in.read();
final int ch4 = in.read();
if ((ch1 | ch2 | ch3 | ch4) < 0) {
throw new EOFException();
}
return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
} else {
return (firstValue << 8) + secondValue;
}
}
private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
int bytesRead;
int totalBytesRead = 0;
while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
totalBytesRead += bytesRead;
}
if (totalBytesRead != length) {
throw new EOFException();
}
}
@Override
public Long getRecordIdentifier(final RepositoryRecord record) {
return record.getCurrent().getId();
}
@Override
public UpdateType getUpdateType(final RepositoryRecord record) {
switch (record.getType()) {
case CONTENTMISSING:
case DELETE:
return UpdateType.DELETE;
case CREATE:
return UpdateType.CREATE;
case UPDATE:
return UpdateType.UPDATE;
case SWAP_OUT:
return UpdateType.SWAP_OUT;
case SWAP_IN:
return UpdateType.SWAP_IN;
}
return null;
}
@Override
public int getVersion() {
return CURRENT_ENCODING_VERSION;
}
@Override
public String getLocation(final RepositoryRecord record) {
return record.getSwapLocation();
}
}
}

View File

@ -0,0 +1,517 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.flowfile.FlowFile;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.wali.SerDe;
import org.wali.UpdateType;
public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
private static final Logger logger = LoggerFactory.getLogger(WriteAheadRepositoryRecordSerde.class);
private static final int CURRENT_ENCODING_VERSION = 9;
public static final byte ACTION_CREATE = 0;
public static final byte ACTION_UPDATE = 1;
public static final byte ACTION_DELETE = 2;
public static final byte ACTION_SWAPPED_OUT = 3;
public static final byte ACTION_SWAPPED_IN = 4;
private long recordsRestored = 0L;
private final ResourceClaimManager claimManager;
public WriteAheadRepositoryRecordSerde(final ResourceClaimManager claimManager) {
this.claimManager = claimManager;
}
@Override
public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out) throws IOException {
serializeEdit(previousRecordState, record, out, false);
}
public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten) throws IOException {
if (record.isMarkedForAbort()) {
logger.warn("Repository Record {} is marked to be aborted; it will be persisted in the FlowFileRepository as a DELETE record", record);
out.write(ACTION_DELETE);
out.writeLong(getRecordIdentifier(record));
serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
return;
}
final UpdateType updateType = getUpdateType(record);
if (updateType.equals(UpdateType.DELETE)) {
out.write(ACTION_DELETE);
out.writeLong(getRecordIdentifier(record));
serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
return;
}
// If there's a Destination Connection, that's the one that we want to associated with this record.
// However, on restart, we will restore the FlowFile and set this connection to its "originalConnection".
// If we then serialize the FlowFile again before it's transferred, it's important to allow this to happen,
// so we use the originalConnection instead
FlowFileQueue associatedQueue = record.getDestination();
if (associatedQueue == null) {
associatedQueue = record.getOriginalQueue();
}
if (updateType.equals(UpdateType.SWAP_OUT)) {
out.write(ACTION_SWAPPED_OUT);
out.writeLong(getRecordIdentifier(record));
out.writeUTF(associatedQueue.getIdentifier());
out.writeUTF(getLocation(record));
return;
}
final FlowFile flowFile = record.getCurrent();
final ContentClaim claim = record.getCurrentClaim();
switch (updateType) {
case UPDATE:
out.write(ACTION_UPDATE);
break;
case CREATE:
out.write(ACTION_CREATE);
break;
case SWAP_IN:
out.write(ACTION_SWAPPED_IN);
break;
default:
throw new AssertionError();
}
out.writeLong(getRecordIdentifier(record));
out.writeLong(flowFile.getEntryDate());
out.writeLong(flowFile.getLineageStartDate());
out.writeLong(flowFile.getLineageStartIndex());
final Long queueDate = flowFile.getLastQueueDate();
out.writeLong(queueDate == null ? System.currentTimeMillis() : queueDate);
out.writeLong(flowFile.getQueueDateIndex());
out.writeLong(flowFile.getSize());
if (associatedQueue == null) {
logger.warn("{} Repository Record {} has no Connection associated with it; it will be destroyed on restart",
new Object[] {this, record});
writeString("", out);
} else {
writeString(associatedQueue.getIdentifier(), out);
}
serializeContentClaim(claim, record.getCurrentClaimOffset(), out);
if (forceAttributesWritten || record.isAttributesChanged() || updateType == UpdateType.CREATE || updateType == UpdateType.SWAP_IN) {
out.write(1); // indicate attributes changed
final Map<String, String> attributes = flowFile.getAttributes();
out.writeInt(attributes.size());
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
writeString(entry.getKey(), out);
writeString(entry.getValue(), out);
}
} else {
out.write(0); // indicate attributes did not change
}
if (updateType == UpdateType.SWAP_IN) {
out.writeUTF(record.getSwapLocation());
}
}
@Override
public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
final int action = in.read();
final long recordId = in.readLong();
if (action == ACTION_DELETE) {
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
if (version > 4) {
deserializeClaim(in, version, ffBuilder);
}
final FlowFileRecord flowFileRecord = ffBuilder.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
record.markForDelete();
return record;
}
if (action == ACTION_SWAPPED_OUT) {
final String queueId = in.readUTF();
final String location = in.readUTF();
final FlowFileQueue queue = getFlowFileQueue(queueId);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(recordId)
.build();
return new StandardRepositoryRecord(queue, flowFileRecord, location);
}
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final RepositoryRecord record = currentRecordStates.get(recordId);
ffBuilder.id(recordId);
if (record != null) {
ffBuilder.fromFlowFile(record.getCurrent());
}
ffBuilder.entryDate(in.readLong());
if (version > 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
}
}
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (version > 7) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (version > 5) {
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (version > 7) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
ffBuilder.size(in.readLong());
final String connectionId = readString(in);
logger.debug("{} -> {}", new Object[] {recordId, connectionId});
deserializeClaim(in, version, ffBuilder);
// recover new attributes, if they changed
final int attributesChanged = in.read();
if (attributesChanged == -1) {
throw new EOFException();
} else if (attributesChanged == 1) {
final int numAttributes = in.readInt();
final Map<String, String> attributes = new HashMap<>();
for (int i = 0; i < numAttributes; i++) {
final String key = readString(in);
final String value = readString(in);
attributes.put(key, value);
}
ffBuilder.addAttributes(attributes);
} else if (attributesChanged != 0) {
throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+ attributesChanged + " after successfully restoring " + recordsRestored + " records. The FlowFile Repository appears to be corrupt!");
}
final FlowFileRecord flowFile = ffBuilder.build();
String swapLocation = null;
if (action == ACTION_SWAPPED_IN) {
swapLocation = in.readUTF();
}
final FlowFileQueue queue = getFlowFileQueue(connectionId);
final StandardRepositoryRecord standardRepoRecord = new StandardRepositoryRecord(queue, flowFile);
if (swapLocation != null) {
standardRepoRecord.setSwapLocation(swapLocation);
}
if (connectionId.isEmpty()) {
logger.warn("{} does not have a Queue associated with it; this record will be discarded", flowFile);
standardRepoRecord.markForAbort();
} else if (queue == null) {
logger.warn("{} maps to unknown Queue {}; this record will be discarded", flowFile, connectionId);
standardRepoRecord.markForAbort();
}
recordsRestored++;
return standardRepoRecord;
}
@Override
public StandardRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
final int action = in.read();
if (action == -1) {
return null;
}
final long recordId = in.readLong();
if (action == ACTION_DELETE) {
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
if (version > 4) {
deserializeClaim(in, version, ffBuilder);
}
final FlowFileRecord flowFileRecord = ffBuilder.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
record.markForDelete();
return record;
}
// if action was not delete, it must be create/swap in
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final long entryDate = in.readLong();
if (version > 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
}
}
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (version > 7) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (version > 5) {
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (version > 7) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
final long size = in.readLong();
final String connectionId = readString(in);
logger.debug("{} -> {}", new Object[] {recordId, connectionId});
ffBuilder.id(recordId);
ffBuilder.entryDate(entryDate);
ffBuilder.size(size);
deserializeClaim(in, version, ffBuilder);
final int attributesChanged = in.read();
if (attributesChanged == 1) {
final int numAttributes = in.readInt();
final Map<String, String> attributes = new HashMap<>();
for (int i = 0; i < numAttributes; i++) {
final String key = readString(in);
final String value = readString(in);
attributes.put(key, value);
}
ffBuilder.addAttributes(attributes);
} else if (attributesChanged == -1) {
throw new EOFException();
} else if (attributesChanged != 0) {
throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+ attributesChanged + " after successfully restoring " + recordsRestored + " records");
}
final FlowFileRecord flowFile = ffBuilder.build();
String swapLocation = null;
if (action == ACTION_SWAPPED_IN) {
swapLocation = in.readUTF();
}
final StandardRepositoryRecord record;
final FlowFileQueue queue = getFlowFileQueue(connectionId);
record = new StandardRepositoryRecord(queue, flowFile);
if (swapLocation != null) {
record.setSwapLocation(swapLocation);
}
if (connectionId.isEmpty()) {
logger.warn("{} does not have a FlowFile Queue associated with it; this record will be discarded", flowFile);
record.markForAbort();
} else if (queue == null) {
logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId);
record.markForAbort();
}
recordsRestored++;
return record;
}
@Override
public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
serializeEdit(null, record, out, true);
}
private void serializeContentClaim(final ContentClaim claim, final long offset, final DataOutputStream out) throws IOException {
if (claim == null) {
out.write(0);
} else {
out.write(1);
final ResourceClaim resourceClaim = claim.getResourceClaim();
writeString(resourceClaim.getId(), out);
writeString(resourceClaim.getContainer(), out);
writeString(resourceClaim.getSection(), out);
out.writeLong(claim.getOffset());
out.writeLong(claim.getLength());
out.writeLong(offset);
out.writeBoolean(resourceClaim.isLossTolerant());
}
}
private void deserializeClaim(final DataInputStream in, final int serializationVersion, final StandardFlowFileRecord.Builder ffBuilder) throws IOException {
// determine current Content Claim.
final int claimExists = in.read();
if (claimExists == 1) {
final String claimId;
if (serializationVersion < 4) {
claimId = String.valueOf(in.readLong());
} else {
claimId = readString(in);
}
final String container = readString(in);
final String section = readString(in);
final long resourceOffset;
final long resourceLength;
if (serializationVersion < 7) {
resourceOffset = 0L;
resourceLength = -1L;
} else {
resourceOffset = in.readLong();
resourceLength = in.readLong();
}
final long claimOffset = in.readLong();
final boolean lossTolerant;
if (serializationVersion >= 3) {
lossTolerant = in.readBoolean();
} else {
lossTolerant = false;
}
final ResourceClaim resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, false);
final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
contentClaim.setLength(resourceLength);
ffBuilder.contentClaim(contentClaim);
ffBuilder.contentClaimOffset(claimOffset);
} else if (claimExists == -1) {
throw new EOFException();
} else if (claimExists != 0) {
throw new IOException("Claim Existence Qualifier not found in stream; found value: "
+ claimExists + " after successfully restoring " + recordsRestored + " records");
}
}
private void writeString(final String toWrite, final OutputStream out) throws IOException {
final byte[] bytes = toWrite.getBytes("UTF-8");
final int utflen = bytes.length;
if (utflen < 65535) {
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
} else {
out.write(255);
out.write(255);
out.write(utflen >>> 24);
out.write(utflen >>> 16);
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
}
}
private String readString(final InputStream in) throws IOException {
final Integer numBytes = readFieldLength(in);
if (numBytes == null) {
throw new EOFException();
}
final byte[] bytes = new byte[numBytes];
fillBuffer(in, bytes, numBytes);
return new String(bytes, "UTF-8");
}
private Integer readFieldLength(final InputStream in) throws IOException {
final int firstValue = in.read();
final int secondValue = in.read();
if (firstValue < 0) {
return null;
}
if (secondValue < 0) {
throw new EOFException();
}
if (firstValue == 0xff && secondValue == 0xff) {
final int ch1 = in.read();
final int ch2 = in.read();
final int ch3 = in.read();
final int ch4 = in.read();
if ((ch1 | ch2 | ch3 | ch4) < 0) {
throw new EOFException();
}
return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
} else {
return (firstValue << 8) + secondValue;
}
}
private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
int bytesRead;
int totalBytesRead = 0;
while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
totalBytesRead += bytesRead;
}
if (totalBytesRead != length) {
throw new EOFException();
}
}
@Override
public int getVersion() {
return CURRENT_ENCODING_VERSION;
}
}

View File

@ -64,29 +64,6 @@ public class StandardResourceClaim implements ResourceClaim, Comparable<Resource
return section;
}
/**
* Provides the natural ordering for ResourceClaim objects. By default they are sorted by their id, then container, then section
*
* @param other other claim
* @return x such that x <=1 if this is less than other;
* x=0 if this.equals(other);
* x >= 1 if this is greater than other
*/
@Override
public int compareTo(final ResourceClaim other) {
final int idComparison = id.compareTo(other.getId());
if (idComparison != 0) {
return idComparison;
}
final int containerComparison = container.compareTo(other.getContainer());
if (containerComparison != 0) {
return containerComparison;
}
return section.compareTo(other.getSection());
}
@Override
public boolean equals(final Object other) {
if (this == other) {

View File

@ -29,10 +29,9 @@ import org.slf4j.LoggerFactory;
public class StandardResourceClaimManager implements ResourceClaimManager {
private static final ConcurrentMap<ResourceClaim, ClaimCount> claimantCounts = new ConcurrentHashMap<>();
private static final Logger logger = LoggerFactory.getLogger(StandardResourceClaimManager.class);
private static final BlockingQueue<ResourceClaim> destructableClaims = new LinkedBlockingQueue<>(50000);
private final ConcurrentMap<ResourceClaim, ClaimCount> claimantCounts = new ConcurrentHashMap<>();
private final BlockingQueue<ResourceClaim> destructableClaims = new LinkedBlockingQueue<>(50000);
@Override
public ResourceClaim newResourceClaim(final String container, final String section, final String id, final boolean lossTolerant, final boolean writable) {
@ -50,7 +49,7 @@ public class StandardResourceClaimManager implements ResourceClaimManager {
return (count == null) ? null : count.getClaim();
}
private static AtomicInteger getCounter(final ResourceClaim claim) {
private AtomicInteger getCounter(final ResourceClaim claim) {
if (claim == null) {
return null;
}

View File

@ -1,101 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.io;
import java.io.IOException;
import java.io.InputStream;
import java.util.concurrent.atomic.AtomicLong;
public class ByteCountingInputStream extends InputStream {
private final AtomicLong bytesReadHolder;
private final InputStream in;
private long bytesSkipped = 0L;
public ByteCountingInputStream(final InputStream in, final AtomicLong longHolder) {
this.in = in;
this.bytesReadHolder = longHolder;
}
@Override
public int read() throws IOException {
final int fromSuper = in.read();
if (fromSuper >= 0) {
bytesReadHolder.getAndIncrement();
}
return fromSuper;
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
final int fromSuper = in.read(b, off, len);
if (fromSuper >= 0) {
bytesReadHolder.getAndAdd(fromSuper);
}
return fromSuper;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
@Override
public long skip(final long n) throws IOException {
final long skipped = in.skip(n);
bytesSkipped += skipped;
return skipped;
}
@Override
public int available() throws IOException {
return in.available();
}
@Override
public void mark(int readlimit) {
in.mark(readlimit);
}
@Override
public boolean markSupported() {
return in.markSupported();
}
@Override
public void reset() throws IOException {
in.reset();
}
@Override
public void close() throws IOException {
in.close();
}
public long getBytesRead() {
return bytesReadHolder.get();
}
public long getBytesSkipped() {
return bytesSkipped;
}
public long getStreamLocation() {
return getBytesRead() + getBytesSkipped();
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import java.util.List;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
public class ContentClaimFieldMap implements Record {
private final ContentClaim contentClaim;
private final long contentClaimOffset;
private final ResourceClaimFieldMap resourceClaimFieldMap;
private final RecordSchema schema;
public ContentClaimFieldMap(final ContentClaim contentClaim, final long contentClaimOffset, final RecordSchema schema) {
this.contentClaim = contentClaim;
this.contentClaimOffset = contentClaimOffset;
this.schema = schema;
final List<RecordField> resourceClaimFields = schema.getField(ContentClaimSchema.RESOURCE_CLAIM).getSubFields();
final RecordSchema resourceClaimSchema = new RecordSchema(resourceClaimFields);
this.resourceClaimFieldMap = new ResourceClaimFieldMap(contentClaim.getResourceClaim(), resourceClaimSchema);
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case ContentClaimSchema.RESOURCE_CLAIM:
return resourceClaimFieldMap;
case ContentClaimSchema.CONTENT_CLAIM_LENGTH:
return contentClaim.getLength();
case ContentClaimSchema.CONTENT_CLAIM_OFFSET:
return contentClaimOffset;
case ContentClaimSchema.RESOURCE_CLAIM_OFFSET:
return contentClaim.getOffset();
default:
return null;
}
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public String toString() {
return "ContentClaimFieldMap[" + contentClaim + "]";
}
public static ContentClaim getContentClaim(final Record claimRecord, final ResourceClaimManager resourceClaimManager) {
final Record resourceClaimRecord = (Record) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM);
final String container = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
final String section = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
final String identifier = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
final Boolean lossTolerant = (Boolean) resourceClaimRecord.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
final Long length = (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_LENGTH);
final Long resourceOffset = (Long) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM_OFFSET);
final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
contentClaim.setLength(length);
return contentClaim;
}
public static Long getContentClaimOffset(final Record claimRecord) {
return (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_OFFSET);
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SimpleRecordField;
public class ContentClaimSchema {
// resource claim fields
public static final String CLAIM_CONTAINER = "Container";
public static final String CLAIM_SECTION = "Section";
public static final String CLAIM_IDENTIFIER = "Identifier";
public static final String LOSS_TOLERANT = "Loss Tolerant";
public static final String RESOURCE_CLAIM = "Resource Claim";
// content claim fields
public static final String RESOURCE_CLAIM_OFFSET = "Resource Claim Offset"; // offset into resource claim where the content claim begins
public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset"; // offset into the content claim where the flowfile begins
public static final String CONTENT_CLAIM_LENGTH = "Content Claim Length";
public static final RecordSchema CONTENT_CLAIM_SCHEMA_V1;
public static final RecordSchema RESOURCE_CLAIM_SCHEMA_V1;
static {
final List<RecordField> resourceClaimFields = new ArrayList<>();
resourceClaimFields.add(new SimpleRecordField(CLAIM_CONTAINER, FieldType.STRING, Repetition.EXACTLY_ONE));
resourceClaimFields.add(new SimpleRecordField(CLAIM_SECTION, FieldType.STRING, Repetition.EXACTLY_ONE));
resourceClaimFields.add(new SimpleRecordField(CLAIM_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
resourceClaimFields.add(new SimpleRecordField(LOSS_TOLERANT, FieldType.BOOLEAN, Repetition.EXACTLY_ONE));
RESOURCE_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(resourceClaimFields));
final List<RecordField> contentClaimFields = new ArrayList<>();
contentClaimFields.add(new ComplexRecordField(RESOURCE_CLAIM, Repetition.EXACTLY_ONE, resourceClaimFields));
contentClaimFields.add(new SimpleRecordField(RESOURCE_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_LENGTH, FieldType.LONG, Repetition.EXACTLY_ONE));
CONTENT_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(contentClaimFields));
}
}

View File

@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import java.util.Map;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
public class FlowFileRecordFieldMap implements Record {
private final FlowFileRecord flowFile;
private final RecordSchema schema;
private final RecordSchema contentClaimSchema;
private final ContentClaimFieldMap contentClaim;
public FlowFileRecordFieldMap(final FlowFileRecord flowFile, final RecordSchema schema) {
this.flowFile = flowFile;
this.schema = schema;
final RecordField contentClaimField = schema.getField(FlowFileSchema.CONTENT_CLAIM);
contentClaimSchema = new RecordSchema(contentClaimField.getSubFields());
contentClaim = flowFile.getContentClaim() == null ? null : new ContentClaimFieldMap(flowFile.getContentClaim(), flowFile.getContentClaimOffset(), contentClaimSchema);
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case FlowFileSchema.ATTRIBUTES:
return flowFile.getAttributes();
case FlowFileSchema.CONTENT_CLAIM:
return contentClaim;
case FlowFileSchema.ENTRY_DATE:
return flowFile.getEntryDate();
case FlowFileSchema.FLOWFILE_SIZE:
return flowFile.getSize();
case FlowFileSchema.LINEAGE_START_DATE:
return flowFile.getLineageStartDate();
case FlowFileSchema.LINEAGE_START_INDEX:
return flowFile.getLineageStartIndex();
case FlowFileSchema.QUEUE_DATE:
return flowFile.getLastQueueDate();
case FlowFileSchema.QUEUE_DATE_INDEX:
return flowFile.getQueueDateIndex();
case FlowFileSchema.RECORD_ID:
return flowFile.getId();
}
return null;
}
@SuppressWarnings("unchecked")
public static FlowFileRecord getFlowFile(final Record record, final ResourceClaimManager claimManager) {
final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder();
builder.id((Long) record.getFieldValue(FlowFileSchema.RECORD_ID));
builder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
builder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
builder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
builder.lineageStart((Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE), (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX));
builder.lastQueued((Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE), (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX));
final Record contentClaimRecord = (Record) record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
if (contentClaimRecord != null) {
final ContentClaim claim = ContentClaimFieldMap.getContentClaim(contentClaimRecord, claimManager);
builder.contentClaim(claim);
final Long offset = ContentClaimFieldMap.getContentClaimOffset(contentClaimRecord);
if (offset != null) {
builder.contentClaimOffset(offset);
}
}
return builder.build();
}
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.MapRecordField;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SimpleRecordField;
public class FlowFileSchema {
public static final String RECORD_ID = "Record ID";
public static final String ENTRY_DATE = "Entry Date";
public static final String LINEAGE_START_DATE = "Lineage Start Date";
public static final String LINEAGE_START_INDEX = "Lineage Start Index";
public static final String QUEUE_DATE = "Queued Date";
public static final String QUEUE_DATE_INDEX = "Queued Date Index";
public static final String FLOWFILE_SIZE = "FlowFile Size";
public static final String CONTENT_CLAIM = "Content Claim";
public static final String ATTRIBUTES = "Attributes";
// attribute fields
public static final String ATTRIBUTE_NAME = "Attribute Name";
public static final String ATTRIBUTE_VALUE = "Attribute Value";
public static final RecordSchema FLOWFILE_SCHEMA_V1;
static {
final List<RecordField> flowFileFields = new ArrayList<>();
final RecordField attributeNameField = new SimpleRecordField(ATTRIBUTE_NAME, FieldType.STRING, Repetition.EXACTLY_ONE);
final RecordField attributeValueField = new SimpleRecordField(ATTRIBUTE_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE);
flowFileFields.add(new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(ENTRY_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(LINEAGE_START_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(LINEAGE_START_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(QUEUE_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(QUEUE_DATE_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE));
flowFileFields.add(new ComplexRecordField(CONTENT_CLAIM, Repetition.ZERO_OR_ONE, ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1.getFields()));
flowFileFields.add(new MapRecordField(ATTRIBUTES, attributeNameField, attributeValueField, Repetition.ZERO_OR_ONE));
FLOWFILE_SCHEMA_V1 = new RecordSchema(flowFileFields);
}
}

View File

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.RepositoryRecord;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
public class RepositoryRecordFieldMap implements Record {
private final RepositoryRecord record;
private final FlowFileRecord flowFile;
private final RecordSchema schema;
private final RecordSchema contentClaimSchema;
public RepositoryRecordFieldMap(final RepositoryRecord record, final RecordSchema repoRecordSchema, final RecordSchema contentClaimSchema) {
this.schema = repoRecordSchema;
this.contentClaimSchema = contentClaimSchema;
this.record = record;
this.flowFile = record.getCurrent();
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case RepositoryRecordSchema.ACTION_TYPE:
return record.getType().name();
case RepositoryRecordSchema.RECORD_ID:
return record.getCurrent().getId();
case RepositoryRecordSchema.SWAP_LOCATION:
return record.getSwapLocation();
case FlowFileSchema.ATTRIBUTES:
return flowFile.getAttributes();
case FlowFileSchema.ENTRY_DATE:
return flowFile.getEntryDate();
case FlowFileSchema.FLOWFILE_SIZE:
return flowFile.getSize();
case FlowFileSchema.LINEAGE_START_DATE:
return flowFile.getLineageStartDate();
case FlowFileSchema.LINEAGE_START_INDEX:
return flowFile.getLineageStartIndex();
case FlowFileSchema.QUEUE_DATE:
return flowFile.getLastQueueDate();
case FlowFileSchema.QUEUE_DATE_INDEX:
return flowFile.getQueueDateIndex();
case FlowFileSchema.CONTENT_CLAIM:
final ContentClaimFieldMap contentClaimFieldMap = record.getCurrentClaim() == null ? null
: new ContentClaimFieldMap(record.getCurrentClaim(), record.getCurrentClaimOffset(), contentClaimSchema);
return contentClaimFieldMap;
case RepositoryRecordSchema.QUEUE_IDENTIFIER:
final FlowFileQueue queue = record.getDestination() == null ? record.getOriginalQueue() : record.getDestination();
return queue.getIdentifier();
default:
return null;
}
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public String toString() {
return "RepositoryRecordFieldMap[" + record + "]";
}
}

View File

@ -0,0 +1,93 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SimpleRecordField;
import org.apache.nifi.repository.schema.UnionRecordField;
public class RepositoryRecordSchema {
public static final String REPOSITORY_RECORD_UPDATE_V1 = "Repository Record Update"; // top level field name
// repository record fields
public static final String ACTION_TYPE = "Action";
public static final String RECORD_ID = "Record ID";
public static final String QUEUE_IDENTIFIER = "Queue Identifier";
public static final String SWAP_LOCATION = "Swap Location";
// Update types
public static final String CREATE_OR_UPDATE_ACTION = "Create or Update";
public static final String DELETE_ACTION = "Delete";
public static final String SWAP_IN_ACTION = "Swap In";
public static final String SWAP_OUT_ACTION = "Swap Out";
public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V1;
public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V1;
public static final RecordSchema DELETE_SCHEMA_V1;
public static final RecordSchema SWAP_IN_SCHEMA_V1;
public static final RecordSchema SWAP_OUT_SCHEMA_V1;
public static final RecordField ACTION_TYPE_FIELD = new SimpleRecordField(ACTION_TYPE, FieldType.STRING, Repetition.EXACTLY_ONE);
public static final RecordField RECORD_ID_FIELD = new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE);
static {
// Fields for "Create" or "Update" records
final List<RecordField> createOrUpdateFields = new ArrayList<>();
createOrUpdateFields.add(ACTION_TYPE_FIELD);
createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields());
createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE));
final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields);
CREATE_OR_UPDATE_SCHEMA_V1 = new RecordSchema(createOrUpdateFields);
// Fields for "Delete" records
final List<RecordField> deleteFields = new ArrayList<>();
deleteFields.add(ACTION_TYPE_FIELD);
deleteFields.add(RECORD_ID_FIELD);
final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields);
DELETE_SCHEMA_V1 = new RecordSchema(deleteFields);
// Fields for "Swap Out" records
final List<RecordField> swapOutFields = new ArrayList<>();
swapOutFields.add(ACTION_TYPE_FIELD);
swapOutFields.add(RECORD_ID_FIELD);
swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields);
SWAP_OUT_SCHEMA_V1 = new RecordSchema(swapOutFields);
// Fields for "Swap In" records
final List<RecordField> swapInFields = new ArrayList<>(createOrUpdateFields);
swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields);
SWAP_IN_SCHEMA_V1 = new RecordSchema(swapInFields);
// Union Field that creates the top-level field type
final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V1, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn);
REPOSITORY_RECORD_SCHEMA_V1 = new RecordSchema(Collections.singletonList(repoUpdateField));
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import org.apache.nifi.repository.schema.NamedValue;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
import org.wali.UpdateType;
public class RepositoryRecordUpdate implements Record {
private final RecordSchema schema;
private final RepositoryRecordFieldMap fieldMap;
public RepositoryRecordUpdate(final RepositoryRecordFieldMap fieldMap, final RecordSchema schema) {
this.schema = schema;
this.fieldMap = fieldMap;
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object getFieldValue(final String fieldName) {
if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V1.equals(fieldName)) {
final String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE);
final UpdateType updateType = UpdateType.valueOf(actionType);
final String actionName;
switch (updateType) {
case CREATE:
case UPDATE:
actionName = RepositoryRecordSchema.CREATE_OR_UPDATE_ACTION;
break;
case DELETE:
actionName = RepositoryRecordSchema.DELETE_ACTION;
break;
case SWAP_IN:
actionName = RepositoryRecordSchema.SWAP_IN_ACTION;
break;
case SWAP_OUT:
actionName = RepositoryRecordSchema.SWAP_OUT_ACTION;
break;
default:
return null;
}
return new NamedValue(actionName, fieldMap);
}
return null;
}
}

View File

@ -0,0 +1,85 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.schema;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
public class ResourceClaimFieldMap implements Record {
private final ResourceClaim resourceClaim;
private final RecordSchema schema;
public ResourceClaimFieldMap(final ResourceClaim resourceClaim, final RecordSchema schema) {
this.resourceClaim = resourceClaim;
this.schema = schema;
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case ContentClaimSchema.CLAIM_CONTAINER:
return resourceClaim.getContainer();
case ContentClaimSchema.CLAIM_SECTION:
return resourceClaim.getSection();
case ContentClaimSchema.CLAIM_IDENTIFIER:
return resourceClaim.getId();
case ContentClaimSchema.LOSS_TOLERANT:
return resourceClaim.isLossTolerant();
}
return null;
}
public static ResourceClaim getResourceClaim(final Record record, final ResourceClaimManager claimManager) {
final String container = (String) record.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
final String section = (String) record.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
final String identifier = (String) record.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
final Boolean lossTolerant = (Boolean) record.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
return claimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
}
@Override
public int hashCode() {
return 41 + 91 * resourceClaim.hashCode();
}
@Override
public boolean equals(final Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (obj.getClass() != ResourceClaimFieldMap.class) {
return false;
}
final ResourceClaimFieldMap other = (ResourceClaimFieldMap) obj;
return resourceClaim.equals(other.resourceClaim);
}
}

View File

@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.schema.FlowFileRecordFieldMap;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SchemaRecordReader;
public class SchemaSwapDeserializer implements SwapDeserializer {
@Override
@SuppressWarnings("unchecked")
public SwapContents deserializeFlowFiles(final DataInputStream in, final String swapLocation, final FlowFileQueue queue, final ResourceClaimManager claimManager) throws IOException {
final RecordSchema schema = RecordSchema.readFrom(in);
final SchemaRecordReader reader = SchemaRecordReader.fromSchema(schema);
final Record parentRecord = reader.readRecord(in);
final List<Record> flowFileRecords = (List<Record>) parentRecord.getFieldValue(SwapSchema.FLOWFILE_CONTENTS);
final List<FlowFileRecord> flowFiles = new ArrayList<>(flowFileRecords.size());
for (final Record record : flowFileRecords) {
flowFiles.add(FlowFileRecordFieldMap.getFlowFile(record, claimManager));
}
final Record summaryRecord = (Record) parentRecord.getFieldValue(SwapSchema.SWAP_SUMMARY);
final SwapSummary swapSummary = SwapSummaryFieldMap.getSwapSummary(summaryRecord, claimManager);
return new StandardSwapContents(swapSummary, flowFiles);
}
@Override
public SwapSummary getSwapSummary(final DataInputStream in, final String swapLocation, final ResourceClaimManager claimManager) throws IOException {
final RecordSchema schema = RecordSchema.readFrom(in);
final List<RecordField> summaryFields = schema.getField(SwapSchema.SWAP_SUMMARY).getSubFields();
final RecordField summaryRecordField = new ComplexRecordField(SwapSchema.SWAP_SUMMARY, Repetition.EXACTLY_ONE, summaryFields);
final RecordSchema summarySchema = new RecordSchema(Collections.singletonList(summaryRecordField));
final Record summaryRecordParent = SchemaRecordReader.fromSchema(summarySchema).readRecord(in);
final Record summaryRecord = (Record) summaryRecordParent.getFieldValue(SwapSchema.SWAP_SUMMARY);
final SwapSummary swapSummary = SwapSummaryFieldMap.getSwapSummary(summaryRecord, claimManager);
return swapSummary;
}
public static String getSerializationName() {
return SchemaSwapSerializer.SERIALIZATION_NAME;
}
}

View File

@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.schema.FlowFileRecordFieldMap;
import org.apache.nifi.controller.repository.schema.FlowFileSchema;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldMapRecord;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SchemaRecordWriter;
import org.apache.nifi.repository.schema.SimpleRecordField;
public class SchemaSwapSerializer implements SwapSerializer {
static final String SERIALIZATION_NAME = "Schema Swap Serialization";
private final RecordSchema schema = SwapSchema.FULL_SWAP_FILE_SCHEMA_V1;
private final RecordSchema flowFileSchema = new RecordSchema(schema.getField(SwapSchema.FLOWFILE_CONTENTS).getSubFields());
@Override
public void serializeFlowFiles(final List<FlowFileRecord> toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream out) throws IOException {
schema.writeTo(out);
long contentSize = 0L;
long maxFlowFileId = -1L;
final List<ResourceClaim> resourceClaims = new ArrayList<>();
for (final FlowFileRecord flowFile : toSwap) {
contentSize += flowFile.getSize();
if (flowFile.getId() > maxFlowFileId) {
maxFlowFileId = flowFile.getId();
}
final ContentClaim contentClaim = flowFile.getContentClaim();
if (contentClaim != null) {
resourceClaims.add(contentClaim.getResourceClaim());
}
}
final QueueSize queueSize = new QueueSize(toSwap.size(), contentSize);
final SwapSummary swapSummary = new StandardSwapSummary(queueSize, maxFlowFileId, resourceClaims);
final Record summaryRecord = new SwapSummaryFieldMap(swapSummary, queue.getIdentifier(), SwapSchema.SWAP_SUMMARY_SCHEMA_V1);
final List<Record> flowFileRecords = toSwap.stream()
.map(flowFile -> new FlowFileRecordFieldMap(flowFile, flowFileSchema))
.collect(Collectors.toList());
// Create a simple record to hold the summary and the flowfile contents
final RecordField summaryField = new SimpleRecordField(SwapSchema.SWAP_SUMMARY, FieldType.COMPLEX, Repetition.EXACTLY_ONE);
final RecordField contentsField = new ComplexRecordField(SwapSchema.FLOWFILE_CONTENTS, Repetition.ZERO_OR_MORE, FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields());
final List<RecordField> fields = new ArrayList<>(2);
fields.add(summaryField);
fields.add(contentsField);
final Map<RecordField, Object> swapFileMap = new LinkedHashMap<>();
swapFileMap.put(summaryField, summaryRecord);
swapFileMap.put(contentsField, flowFileRecords);
final Record swapFileRecord = new FieldMapRecord(swapFileMap, new RecordSchema(fields));
final SchemaRecordWriter writer = new SchemaRecordWriter();
writer.writeRecord(swapFileRecord, out);
out.flush();
}
@Override
public String getSerializationName() {
return SERIALIZATION_NAME;
}
}

View File

@ -0,0 +1,303 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.IncompleteSwapFileException;
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SimpleSwapDeserializer implements SwapDeserializer {
public static final int SWAP_ENCODING_VERSION = 10;
private static final Logger logger = LoggerFactory.getLogger(SimpleSwapDeserializer.class);
@Override
public SwapSummary getSwapSummary(final DataInputStream in, final String swapLocation, final ResourceClaimManager claimManager) throws IOException {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
final String errMsg = "Cannot swap FlowFiles in from " + swapLocation + " because the encoding version is "
+ swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
throw new IOException(errMsg);
}
final int numRecords;
final long contentSize;
Long maxRecordId = null;
try {
in.readUTF(); // ignore Connection ID
numRecords = in.readInt();
contentSize = in.readLong();
if (numRecords == 0) {
return StandardSwapSummary.EMPTY_SUMMARY;
}
if (swapEncodingVersion > 7) {
maxRecordId = in.readLong();
}
} catch (final EOFException eof) {
logger.warn("Found premature End-of-File when reading Swap File {}. EOF occurred before any FlowFiles were encountered", swapLocation);
return StandardSwapSummary.EMPTY_SUMMARY;
}
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
final SwapContents swapContents = deserializeFlowFiles(in, queueSize, maxRecordId, swapEncodingVersion, claimManager, swapLocation);
return swapContents.getSummary();
}
@Override
public SwapContents deserializeFlowFiles(final DataInputStream in, final String swapLocation, final FlowFileQueue queue, final ResourceClaimManager claimManager) throws IOException {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
throw new IOException("Cannot swap FlowFiles in from SwapFile because the encoding version is "
+ swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)");
}
final String connectionId = in.readUTF(); // Connection ID
if (!connectionId.equals(queue.getIdentifier())) {
throw new IllegalArgumentException("Cannot deserialize FlowFiles from Swap File at location " + swapLocation
+ " because those FlowFiles belong to Connection with ID " + connectionId + " and an attempt was made to swap them into a Connection with ID " + queue.getIdentifier());
}
int numRecords = 0;
long contentSize = 0L;
Long maxRecordId = null;
try {
numRecords = in.readInt();
contentSize = in.readLong(); // Content Size
if (swapEncodingVersion > 7) {
maxRecordId = in.readLong(); // Max Record ID
}
} catch (final EOFException eof) {
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections.emptyList());
final SwapContents partialContents = new StandardSwapContents(summary, Collections.emptyList());
throw new IncompleteSwapFileException(swapLocation, partialContents);
}
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
return deserializeFlowFiles(in, queueSize, maxRecordId, swapEncodingVersion, claimManager, swapLocation);
}
private static SwapContents deserializeFlowFiles(final DataInputStream in, final QueueSize queueSize, final Long maxRecordId,
final int serializationVersion, final ResourceClaimManager claimManager, final String location) throws IOException {
final List<FlowFileRecord> flowFiles = new ArrayList<>(queueSize.getObjectCount());
final List<ResourceClaim> resourceClaims = new ArrayList<>(queueSize.getObjectCount());
Long maxId = maxRecordId;
for (int i = 0; i < queueSize.getObjectCount(); i++) {
try {
// legacy encoding had an "action" because it used to be couple with FlowFile Repository code
if (serializationVersion < 3) {
final int action = in.read();
if (action != 1) {
throw new IOException("Swap File is version " + serializationVersion + " but did not contain a 'UPDATE' record type");
}
}
final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
final long recordId = in.readLong();
if (maxId == null || recordId > maxId) {
maxId = recordId;
}
ffBuilder.id(recordId);
ffBuilder.entryDate(in.readLong());
if (serializationVersion > 1) {
// Lineage information was added in version 2
if (serializationVersion < 10) {
final int numLineageIdentifiers = in.readInt();
for (int lineageIdIdx = 0; lineageIdIdx < numLineageIdentifiers; lineageIdIdx++) {
in.readUTF(); //skip each identifier
}
}
// version 9 adds in a 'lineage start index'
final long lineageStartDate = in.readLong();
final long lineageStartIndex;
if (serializationVersion > 8) {
lineageStartIndex = in.readLong();
} else {
lineageStartIndex = 0L;
}
ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
if (serializationVersion > 5) {
// Version 9 adds in a 'queue date index'
final long lastQueueDate = in.readLong();
final long queueDateIndex;
if (serializationVersion > 8) {
queueDateIndex = in.readLong();
} else {
queueDateIndex = 0L;
}
ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
}
}
ffBuilder.size(in.readLong());
if (serializationVersion < 3) {
readString(in); // connection Id
}
final boolean hasClaim = in.readBoolean();
ResourceClaim resourceClaim = null;
if (hasClaim) {
final String claimId;
if (serializationVersion < 5) {
claimId = String.valueOf(in.readLong());
} else {
claimId = in.readUTF();
}
final String container = in.readUTF();
final String section = in.readUTF();
final long resourceOffset;
final long resourceLength;
if (serializationVersion < 6) {
resourceOffset = 0L;
resourceLength = -1L;
} else {
resourceOffset = in.readLong();
resourceLength = in.readLong();
}
final long claimOffset = in.readLong();
final boolean lossTolerant;
if (serializationVersion >= 4) {
lossTolerant = in.readBoolean();
} else {
lossTolerant = false;
}
resourceClaim = claimManager.getResourceClaim(container, section, claimId);
if (resourceClaim == null) {
logger.error("Swap file indicates that FlowFile was referencing Resource Claim at container={}, section={}, claimId={}, "
+ "but this Resource Claim cannot be found! Will create a temporary Resource Claim, but this may affect the framework's "
+ "ability to properly clean up this resource", container, section, claimId);
resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, true);
}
final StandardContentClaim claim = new StandardContentClaim(resourceClaim, resourceOffset);
claim.setLength(resourceLength);
ffBuilder.contentClaim(claim);
ffBuilder.contentClaimOffset(claimOffset);
}
boolean attributesChanged = true;
if (serializationVersion < 3) {
attributesChanged = in.readBoolean();
}
if (attributesChanged) {
final int numAttributes = in.readInt();
for (int j = 0; j < numAttributes; j++) {
final String key = readString(in);
final String value = readString(in);
ffBuilder.addAttribute(key, value);
}
}
final FlowFileRecord record = ffBuilder.build();
if (resourceClaim != null) {
resourceClaims.add(resourceClaim);
}
flowFiles.add(record);
} catch (final EOFException eof) {
final SwapSummary swapSummary = new StandardSwapSummary(queueSize, maxId, resourceClaims);
final SwapContents partialContents = new StandardSwapContents(swapSummary, flowFiles);
throw new IncompleteSwapFileException(location, partialContents);
}
}
final SwapSummary swapSummary = new StandardSwapSummary(queueSize, maxId, resourceClaims);
return new StandardSwapContents(swapSummary, flowFiles);
}
private static String readString(final InputStream in) throws IOException {
final Integer numBytes = readFieldLength(in);
if (numBytes == null) {
throw new EOFException();
}
final byte[] bytes = new byte[numBytes];
fillBuffer(in, bytes, numBytes);
return new String(bytes, StandardCharsets.UTF_8);
}
private static Integer readFieldLength(final InputStream in) throws IOException {
final int firstValue = in.read();
final int secondValue = in.read();
if (firstValue < 0) {
return null;
}
if (secondValue < 0) {
throw new EOFException();
}
if (firstValue == 0xff && secondValue == 0xff) {
final int ch1 = in.read();
final int ch2 = in.read();
final int ch3 = in.read();
final int ch4 = in.read();
if ((ch1 | ch2 | ch3 | ch4) < 0) {
throw new EOFException();
}
return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
} else {
return (firstValue << 8) + secondValue;
}
}
private static void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
int bytesRead;
int totalBytesRead = 0;
while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
totalBytesRead += bytesRead;
}
if (totalBytesRead != length) {
throw new EOFException();
}
}
}

View File

@ -0,0 +1,133 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* @deprecated in favor of using {@link SchemaSwapSerializer}.
*/
@Deprecated
public class SimpleSwapSerializer implements SwapSerializer {
private static final Logger logger = LoggerFactory.getLogger(SimpleSwapSerializer.class);
public static final int SWAP_ENCODING_VERSION = 10;
@Override
public void serializeFlowFiles(final List<FlowFileRecord> toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream destination) throws IOException {
if (toSwap == null || toSwap.isEmpty()) {
return;
}
long contentSize = 0L;
for (final FlowFileRecord record : toSwap) {
contentSize += record.getSize();
}
// persist record to disk via the swap file
final DataOutputStream out = new DataOutputStream(destination);
try {
out.writeInt(SWAP_ENCODING_VERSION);
out.writeUTF(queue.getIdentifier());
out.writeInt(toSwap.size());
out.writeLong(contentSize);
// get the max record id and write that out so that we know it quickly for restoration
long maxRecordId = 0L;
for (final FlowFileRecord flowFile : toSwap) {
if (flowFile.getId() > maxRecordId) {
maxRecordId = flowFile.getId();
}
}
out.writeLong(maxRecordId);
for (final FlowFileRecord flowFile : toSwap) {
out.writeLong(flowFile.getId());
out.writeLong(flowFile.getEntryDate());
out.writeLong(flowFile.getLineageStartDate());
out.writeLong(flowFile.getLineageStartIndex());
out.writeLong(flowFile.getLastQueueDate());
out.writeLong(flowFile.getQueueDateIndex());
out.writeLong(flowFile.getSize());
final ContentClaim claim = flowFile.getContentClaim();
if (claim == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
final ResourceClaim resourceClaim = claim.getResourceClaim();
out.writeUTF(resourceClaim.getId());
out.writeUTF(resourceClaim.getContainer());
out.writeUTF(resourceClaim.getSection());
out.writeLong(claim.getOffset());
out.writeLong(claim.getLength());
out.writeLong(flowFile.getContentClaimOffset());
out.writeBoolean(resourceClaim.isLossTolerant());
}
final Map<String, String> attributes = flowFile.getAttributes();
out.writeInt(attributes.size());
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
writeString(entry.getKey(), out);
writeString(entry.getValue(), out);
}
}
} finally {
out.flush();
}
logger.info("Successfully swapped out {} FlowFiles from {} to Swap File {}", toSwap.size(), queue, swapLocation);
}
private void writeString(final String toWrite, final OutputStream out) throws IOException {
final byte[] bytes = toWrite.getBytes(StandardCharsets.UTF_8);
final int utflen = bytes.length;
if (utflen < 65535) {
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
} else {
out.write(255);
out.write(255);
out.write(utflen >>> 24);
out.write(utflen >>> 16);
out.write(utflen >>> 8);
out.write(utflen);
out.write(bytes);
}
}
@Override
public String getSerializationName() {
return "Simple Swap Serializer";
}
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.DataInputStream;
import java.io.IOException;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
public interface SwapDeserializer {
SwapContents deserializeFlowFiles(DataInputStream in, String swapLocation, FlowFileQueue queue, ResourceClaimManager claimManager) throws IOException;
SwapSummary getSwapSummary(DataInputStream in, String swapLocation, ResourceClaimManager claimManager) throws IOException;
}

View File

@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.controller.repository.schema.ContentClaimSchema;
import org.apache.nifi.controller.repository.schema.FlowFileSchema;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.MapRecordField;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SimpleRecordField;
public class SwapSchema {
public static final RecordSchema SWAP_SUMMARY_SCHEMA_V1;
public static final RecordSchema SWAP_CONTENTS_SCHEMA_V1;
public static final RecordSchema FULL_SWAP_FILE_SCHEMA_V1;
public static final String RESOURCE_CLAIMS = "Resource Claims";
public static final String RESOURCE_CLAIM = "Resource Claim";
public static final String RESOURCE_CLAIM_COUNT = "Claim Count";
public static final String QUEUE_IDENTIFIER = "Queue Identifier";
public static final String FLOWFILE_COUNT = "FlowFile Count";
public static final String FLOWFILE_SIZE = "FlowFile Size";
public static final String MAX_RECORD_ID = "Max Record ID";
public static final String SWAP_SUMMARY = "Swap Summary";
public static final String FLOWFILE_CONTENTS = "FlowFiles";
static {
final RecordField queueIdentifier = new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE);
final RecordField flowFileCount = new SimpleRecordField(FLOWFILE_COUNT, FieldType.INT, Repetition.EXACTLY_ONE);
final RecordField flowFileSize = new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE);
final RecordField maxRecordId = new SimpleRecordField(MAX_RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE);
final RecordField resourceClaimField = new ComplexRecordField(RESOURCE_CLAIM, Repetition.EXACTLY_ONE, ContentClaimSchema.RESOURCE_CLAIM_SCHEMA_V1.getFields());
final RecordField claimCountField = new SimpleRecordField(RESOURCE_CLAIM_COUNT, FieldType.INT, Repetition.EXACTLY_ONE);
final RecordField resourceClaims = new MapRecordField(RESOURCE_CLAIMS, resourceClaimField, claimCountField, Repetition.EXACTLY_ONE);
final List<RecordField> summaryFields = new ArrayList<>();
summaryFields.add(queueIdentifier);
summaryFields.add(flowFileCount);
summaryFields.add(flowFileSize);
summaryFields.add(maxRecordId);
summaryFields.add(resourceClaims);
SWAP_SUMMARY_SCHEMA_V1 = new RecordSchema(summaryFields);
final RecordField flowFiles = new ComplexRecordField(FLOWFILE_CONTENTS, Repetition.ZERO_OR_MORE, FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields());
final List<RecordField> contentsFields = Collections.singletonList(flowFiles);
SWAP_CONTENTS_SCHEMA_V1 = new RecordSchema(contentsFields);
final List<RecordField> fullSchemaFields = new ArrayList<>();
fullSchemaFields.add(new ComplexRecordField(SWAP_SUMMARY, Repetition.EXACTLY_ONE, summaryFields));
fullSchemaFields.add(new ComplexRecordField(FLOWFILE_CONTENTS, Repetition.ZERO_OR_MORE, FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields()));
FULL_SWAP_FILE_SCHEMA_V1 = new RecordSchema(fullSchemaFields);
}
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.io.IOException;
import java.io.OutputStream;
import java.util.List;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
public interface SwapSerializer {
void serializeFlowFiles(List<FlowFileRecord> toSwap, FlowFileQueue queue, String swapLocation, OutputStream destination) throws IOException;
String getSerializationName();
}

View File

@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.schema.ResourceClaimFieldMap;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
public class SwapSummaryFieldMap implements Record {
private final SwapSummary swapSummary;
private final RecordSchema schema;
private final String queueIdentifier;
private final Map<ResourceClaimFieldMap, Integer> claimCounts;
public SwapSummaryFieldMap(final SwapSummary summary, final String queueIdentifier, final RecordSchema schema) {
this.swapSummary = summary;
this.queueIdentifier = queueIdentifier;
this.schema = schema;
final RecordField resourceClaimField = schema.getField(SwapSchema.RESOURCE_CLAIMS).getSubFields().get(0);
final RecordSchema resourceClaimSchema = new RecordSchema(resourceClaimField.getSubFields());
final List<ResourceClaim> resourceClaims = summary.getResourceClaims();
claimCounts = new HashMap<>();
for (final ResourceClaim claim : resourceClaims) {
final ResourceClaimFieldMap fieldMap = new ResourceClaimFieldMap(claim, resourceClaimSchema);
final Integer count = claimCounts.get(fieldMap);
if (count == null) {
claimCounts.put(fieldMap, 1);
} else {
claimCounts.put(fieldMap, count + 1);
}
}
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case SwapSchema.MAX_RECORD_ID:
return swapSummary.getMaxFlowFileId();
case SwapSchema.FLOWFILE_COUNT:
return swapSummary.getQueueSize().getObjectCount();
case SwapSchema.FLOWFILE_SIZE:
return swapSummary.getQueueSize().getByteCount();
case SwapSchema.QUEUE_IDENTIFIER:
return queueIdentifier;
case SwapSchema.RESOURCE_CLAIMS:
return claimCounts;
}
return null;
}
@SuppressWarnings("unchecked")
public static SwapSummary getSwapSummary(final Record record, final ResourceClaimManager claimManager) {
final int flowFileCount = (Integer) record.getFieldValue(SwapSchema.FLOWFILE_COUNT);
final long flowFileSize = (Long) record.getFieldValue(SwapSchema.FLOWFILE_SIZE);
final QueueSize queueSize = new QueueSize(flowFileCount, flowFileSize);
final long maxFlowFileId = (Long) record.getFieldValue(SwapSchema.MAX_RECORD_ID);
final Map<Record, Integer> resourceClaimRecords = (Map<Record, Integer>) record.getFieldValue(SwapSchema.RESOURCE_CLAIMS);
final List<ResourceClaim> resourceClaims = new ArrayList<>();
for (final Map.Entry<Record, Integer> entry : resourceClaimRecords.entrySet()) {
final Record resourceClaimRecord = entry.getKey();
final ResourceClaim claim = ResourceClaimFieldMap.getResourceClaim(resourceClaimRecord, claimManager);
for (int i = 0; i < entry.getValue(); i++) {
resourceClaims.add(claim);
}
}
return new StandardSwapSummary(queueSize, maxFlowFileId, resourceClaims);
}
}

View File

@ -22,26 +22,20 @@ import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.SwapManagerInitializationContext;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.events.EventReporter;
import org.junit.Test;
import org.mockito.Mockito;
@ -56,7 +50,9 @@ public class TestFileSystemSwapManager {
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
final SwapContents swapContents = FileSystemSwapManager.deserializeFlowFiles(in, "/src/test/resources/old-swap-file.swap", flowFileQueue, new NopResourceClaimManager());
final FileSystemSwapManager swapManager = createSwapManager();
final SwapContents swapContents = swapManager.peek("src/test/resources/old-swap-file.swap", flowFileQueue);
final List<FlowFileRecord> records = swapContents.getFlowFiles();
assertEquals(10000, records.size());
@ -67,53 +63,32 @@ public class TestFileSystemSwapManager {
}
}
@Test
public void testRoundTripSerializeDeserialize() throws IOException {
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = new TestFlowFile(attrs, i);
toSwap.add(ff);
}
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
private FileSystemSwapManager createSwapManager() {
final FileSystemSwapManager swapManager = new FileSystemSwapManager();
final ResourceClaimManager resourceClaimManager = new NopResourceClaimManager();
final FlowFileRepository flowfileRepo = Mockito.mock(FlowFileRepository.class);
swapManager.initialize(new SwapManagerInitializationContext() {
@Override
public ResourceClaimManager getResourceClaimManager() {
return resourceClaimManager;
}
final String swapLocation = "target/testRoundTrip.swap";
final File swapFile = new File(swapLocation);
Files.deleteIfExists(swapFile.toPath());
@Override
public FlowFileRepository getFlowFileRepository() {
return flowfileRepo;
}
try (final FileOutputStream fos = new FileOutputStream(swapFile)) {
FileSystemSwapManager.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
}
@Override
public EventReporter getEventReporter() {
return EventReporter.NO_OP;
}
});
final SwapContents swappedIn;
try (final FileInputStream fis = new FileInputStream(swapFile);
final DataInputStream dis = new DataInputStream(fis)) {
swappedIn = FileSystemSwapManager.deserializeFlowFiles(dis, swapLocation, flowFileQueue, Mockito.mock(ResourceClaimManager.class));
}
assertEquals(toSwap.size(), swappedIn.getFlowFiles().size());
for (int i = 0; i < toSwap.size(); i++) {
final FlowFileRecord pre = toSwap.get(i);
final FlowFileRecord post = swappedIn.getFlowFiles().get(i);
assertEquals(pre.getSize(), post.getSize());
assertEquals(pre.getAttributes(), post.getAttributes());
assertEquals(pre.getSize(), post.getSize());
assertEquals(pre.getId(), post.getId());
assertEquals(pre.getContentClaim(), post.getContentClaim());
assertEquals(pre.getContentClaimOffset(), post.getContentClaimOffset());
assertEquals(pre.getEntryDate(), post.getEntryDate());
assertEquals(pre.getLastQueueDate(), post.getLastQueueDate());
assertEquals(pre.getLineageStartDate(), post.getLineageStartDate());
assertEquals(pre.getPenaltyExpirationMillis(), post.getPenaltyExpirationMillis());
}
return swapManager;
}
public class NopResourceClaimManager implements ResourceClaimManager {
@Override
public ResourceClaim newResourceClaim(String container, String section, String id, boolean lossTolerant, boolean writable) {
return null;
@ -165,89 +140,4 @@ public class TestFileSystemSwapManager {
}
}
private static class TestFlowFile implements FlowFileRecord {
private static final AtomicLong idGenerator = new AtomicLong(0L);
private final long id = idGenerator.getAndIncrement();
private final long entryDate = System.currentTimeMillis();
private final long lastQueueDate = System.currentTimeMillis();
private final Map<String, String> attributes;
private final long size;
public TestFlowFile(final Map<String, String> attributes, final long size) {
this.attributes = attributes;
this.size = size;
}
@Override
public long getId() {
return id;
}
@Override
public long getEntryDate() {
return entryDate;
}
@Override
public long getLineageStartDate() {
return entryDate;
}
@Override
public Long getLastQueueDate() {
return lastQueueDate;
}
@Override
public boolean isPenalized() {
return false;
}
@Override
public String getAttribute(String key) {
return attributes.get(key);
}
@Override
public long getSize() {
return size;
}
@Override
public Map<String, String> getAttributes() {
return Collections.unmodifiableMap(attributes);
}
@Override
public int compareTo(final FlowFile o) {
return Long.compare(id, o.getId());
}
@Override
public long getPenaltyExpirationMillis() {
return -1L;
}
@Override
public ContentClaim getContentClaim() {
return null;
}
@Override
public long getContentClaimOffset() {
return 0;
}
@Override
public long getLineageStartIndex() {
return 0;
}
@Override
public long getQueueDateIndex() {
return 0;
}
}
}

View File

@ -49,6 +49,7 @@ import org.apache.nifi.controller.swap.StandardSwapContents;
import org.apache.nifi.controller.swap.StandardSwapSummary;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.file.FileUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -64,6 +65,7 @@ public class TestWriteAheadFlowFileRepository {
}
@Before
@After
public void clearRepo() throws IOException {
final File target = new File("target");
final File testRepo = new File(target, "test-repo");

View File

@ -0,0 +1,136 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.flowfile.FlowFile;
public class MockFlowFile implements FlowFileRecord {
private static final AtomicLong idGenerator = new AtomicLong(0L);
private final long id;
private final long entryDate = System.currentTimeMillis();
private final long lastQueueDate = System.currentTimeMillis();
private final Map<String, String> attributes;
private final long size;
private final ContentClaim contentClaim;
public MockFlowFile(final Map<String, String> attributes, final long size, final ResourceClaimManager claimManager) {
this(attributes, size, createContentClaim(String.valueOf(idGenerator.get()), claimManager));
}
public MockFlowFile(final Map<String, String> attributes, final long size, final ContentClaim contentClaim) {
this(idGenerator.getAndIncrement(), attributes, size, contentClaim);
}
public MockFlowFile(final long id, final Map<String, String> attributes, final long size, final ContentClaim contentClaim) {
this.id = id;
this.attributes = new HashMap<>(attributes);
this.size = size;
this.contentClaim = contentClaim;
}
public static ContentClaim createContentClaim(final String id, final ResourceClaimManager claimManager) {
final ResourceClaim resourceClaim = claimManager.newResourceClaim("container", "section", id, false, false);
claimManager.incrementClaimantCount(resourceClaim);
return new StandardContentClaim(resourceClaim, 3L);
}
@Override
public long getId() {
return id;
}
@Override
public long getEntryDate() {
return entryDate;
}
@Override
public long getLineageStartDate() {
return entryDate;
}
@Override
public Long getLastQueueDate() {
return lastQueueDate;
}
@Override
public boolean isPenalized() {
return false;
}
@Override
public String getAttribute(String key) {
return attributes.get(key);
}
@Override
public long getSize() {
return size;
}
@Override
public Map<String, String> getAttributes() {
return Collections.unmodifiableMap(attributes);
}
@Override
public int compareTo(final FlowFile o) {
return Long.compare(id, o.getId());
}
@Override
public long getPenaltyExpirationMillis() {
return -1L;
}
@Override
public ContentClaim getContentClaim() {
return contentClaim;
}
@Override
public long getContentClaimOffset() {
return 1;
}
@Override
public long getLineageStartIndex() {
return 0;
}
@Override
public long getQueueDateIndex() {
return 0;
}
public static void resetIdGenerator() {
idGenerator.set(0L);
}
}

View File

@ -0,0 +1,195 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.stream.io.NullOutputStream;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.mockito.Mockito;
public class TestSchemaSwapSerializerDeserializer {
@Before
public void setup() {
MockFlowFile.resetIdGenerator();
}
@Test
public void testRoundTripSerializeDeserializeSummary() throws IOException {
final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
final ResourceClaim firstResourceClaim = resourceClaimManager.newResourceClaim("container", "section", "id", true, false);
resourceClaimManager.incrementClaimantCount(firstResourceClaim);
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
long size = 0L;
final ContentClaim firstClaim = MockFlowFile.createContentClaim("id", resourceClaimManager);
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = i < 2 ? new MockFlowFile(attrs, i, firstClaim) : new MockFlowFile(attrs, i, resourceClaimManager);
toSwap.add(ff);
size += i;
}
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
final String swapLocation = "target/testRoundTrip.swap";
final File swapFile = new File(swapLocation);
Files.deleteIfExists(swapFile.toPath());
final SwapSerializer serializer = new SchemaSwapSerializer();
try (final FileOutputStream fos = new FileOutputStream(swapFile)) {
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
}
final SwapDeserializer deserializer = new SchemaSwapDeserializer();
final SwapSummary swapSummary;
try (final FileInputStream fis = new FileInputStream(swapFile);
final DataInputStream dis = new DataInputStream(fis)) {
swapSummary = deserializer.getSwapSummary(dis, swapLocation, resourceClaimManager);
}
assertEquals(10000, swapSummary.getQueueSize().getObjectCount());
assertEquals(size, swapSummary.getQueueSize().getByteCount());
assertEquals(9999, swapSummary.getMaxFlowFileId().intValue());
final List<ResourceClaim> resourceClaims = swapSummary.getResourceClaims();
assertEquals(10000, resourceClaims.size());
assertFalse(resourceClaims.stream().anyMatch(claim -> claim == null));
assertEquals(2, resourceClaims.stream().filter(claim -> claim.getId().equals("id")).collect(Collectors.counting()).intValue());
final Set<ResourceClaim> uniqueClaims = new HashSet<>(resourceClaims);
assertEquals(9999, uniqueClaims.size());
}
@Test
public void testRoundTripSerializeDeserializeFullSwapFile() throws IOException, InterruptedException {
final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
long size = 0L;
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = new MockFlowFile(attrs, i, resourceClaimManager);
toSwap.add(ff);
size += i;
}
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
final String swapLocation = "target/testRoundTrip.swap";
final File swapFile = new File(swapLocation);
Files.deleteIfExists(swapFile.toPath());
final SwapSerializer serializer = new SchemaSwapSerializer();
try (final OutputStream fos = new FileOutputStream(swapFile);
final OutputStream out = new BufferedOutputStream(fos)) {
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, out);
}
final SwapContents contents;
final SwapDeserializer deserializer = new SchemaSwapDeserializer();
try (final FileInputStream fis = new FileInputStream(swapFile);
final InputStream bufferedIn = new BufferedInputStream(fis);
final DataInputStream dis = new DataInputStream(bufferedIn)) {
contents = deserializer.deserializeFlowFiles(dis, swapLocation, flowFileQueue, resourceClaimManager);
}
final SwapSummary swapSummary = contents.getSummary();
assertEquals(10000, swapSummary.getQueueSize().getObjectCount());
assertEquals(size, swapSummary.getQueueSize().getByteCount());
assertEquals(9999, swapSummary.getMaxFlowFileId().intValue());
assertEquals(10000, contents.getFlowFiles().size());
int counter = 0;
for (final FlowFileRecord flowFile : contents.getFlowFiles()) {
final int i = counter++;
assertEquals(String.valueOf(i), flowFile.getAttribute("i"));
assertEquals(i, flowFile.getSize());
}
}
@Test
@Ignore("For manual testing, in order to ensure that changes do not negatively impact performance")
public void testWritePerformance() throws IOException, InterruptedException {
final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = new MockFlowFile(attrs, i, resourceClaimManager);
toSwap.add(ff);
}
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
final String swapLocation = "target/testRoundTrip.swap";
final int iterations = 1000;
final long start = System.nanoTime();
final SwapSerializer serializer = new SchemaSwapSerializer();
for (int i = 0; i < iterations; i++) {
try (final OutputStream out = new NullOutputStream()) {
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, out);
}
}
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
System.out.println("Wrote " + iterations + " Swap Files in " + millis + " millis");
}
}

View File

@ -0,0 +1,139 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.swap;
import static org.junit.Assert.assertEquals;
import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.SwapContents;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.stream.io.NullOutputStream;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.mockito.Mockito;
public class TestSimpleSwapSerializerDeserializer {
@Before
public void setup() {
MockFlowFile.resetIdGenerator();
}
@Test
public void testRoundTripSerializeDeserialize() throws IOException {
final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = new MockFlowFile(attrs, i, resourceClaimManager);
toSwap.add(ff);
}
final String queueId = "87bb99fe-412c-49f6-a441-d1b0af4e20b4";
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn(queueId);
final String swapLocation = "target/testRoundTrip-" + queueId + ".swap";
final File swapFile = new File(swapLocation);
Files.deleteIfExists(swapFile.toPath());
try {
final SimpleSwapSerializer serializer = new SimpleSwapSerializer();
try (final FileOutputStream fos = new FileOutputStream(swapFile)) {
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, fos);
}
final SimpleSwapDeserializer deserializer = new SimpleSwapDeserializer();
final SwapContents swappedIn;
try (final FileInputStream fis = new FileInputStream(swapFile);
final DataInputStream dis = new DataInputStream(fis)) {
swappedIn = deserializer.deserializeFlowFiles(dis, swapLocation, flowFileQueue, resourceClaimManager);
}
assertEquals(toSwap.size(), swappedIn.getFlowFiles().size());
for (int i = 0; i < toSwap.size(); i++) {
final FlowFileRecord pre = toSwap.get(i);
final FlowFileRecord post = swappedIn.getFlowFiles().get(i);
assertEquals(pre.getSize(), post.getSize());
assertEquals(pre.getAttributes(), post.getAttributes());
assertEquals(pre.getSize(), post.getSize());
assertEquals(pre.getId(), post.getId());
assertEquals(pre.getContentClaim(), post.getContentClaim());
assertEquals(pre.getContentClaimOffset(), post.getContentClaimOffset());
assertEquals(pre.getEntryDate(), post.getEntryDate());
assertEquals(pre.getLastQueueDate(), post.getLastQueueDate());
assertEquals(pre.getLineageStartDate(), post.getLineageStartDate());
assertEquals(pre.getPenaltyExpirationMillis(), post.getPenaltyExpirationMillis());
}
} finally {
Files.deleteIfExists(swapFile.toPath());
}
}
@Test
@Ignore("For manual testing only. Not intended to be run as part of the automated unit tests but can "
+ "be convenient for determining a baseline for performance if making modifications.")
public void testWritePerformance() throws IOException, InterruptedException {
final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
final Map<String, String> attrs = new HashMap<>();
for (int i = 0; i < 10000; i++) {
attrs.put("i", String.valueOf(i));
final FlowFileRecord ff = new MockFlowFile(attrs, i, resourceClaimManager);
toSwap.add(ff);
}
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
final String swapLocation = "target/testRoundTrip.swap";
final int iterations = 1000;
final long start = System.nanoTime();
final SwapSerializer serializer = new SimpleSwapSerializer();
for (int i = 0; i < iterations; i++) {
try (final OutputStream out = new NullOutputStream()) {
serializer.serializeFlowFiles(toSwap, flowFileQueue, swapLocation, out);
}
}
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
System.out.println("Wrote " + iterations + " Swap Files in " + millis + " millis");
}
}

View File

@ -34,6 +34,10 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-data-provenance-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-schema-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>

View File

@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.TocWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class AbstractRecordWriter implements RecordWriter {
private static final Logger logger = LoggerFactory.getLogger(AbstractRecordWriter.class);
private final File file;
private final TocWriter tocWriter;
private final Lock lock = new ReentrantLock();
private volatile boolean dirty = false;
private volatile boolean closed = false;
private int recordsWritten = 0;
public AbstractRecordWriter(final File file, final TocWriter writer) throws IOException {
logger.trace("Creating Record Writer for {}", file);
this.file = file;
this.tocWriter = writer;
}
@Override
public synchronized void close() throws IOException {
closed = true;
logger.trace("Closing Record Writer for {}", file == null ? null : file.getName());
lock();
try {
flush();
try {
// We want to close 'out' only if the writer is not 'dirty'.
// If the writer is dirty, then there was a failure to write
// to disk, which means that we likely have a partial record written
// to disk.
//
// If we call close() on out, it will in turn call flush() on the underlying
// output stream, which is a BufferedOutputStream. As a result, we will end
// up flushing the buffer after a partially written record, which results in
// essentially random bytes being written to the repository, which causes
// corruption and un-recoverability. Since we will close the underlying 'rawOutStream'
// below, we will still appropriately clean up the resources help by this writer, so
// we are still OK in terms of closing all resources held by the writer.
final OutputStream buffered = getBufferedOutputStream();
if (buffered != null && !isDirty()) {
buffered.close();
}
} finally {
final OutputStream underlying = getUnderlyingOutputStream();
if (underlying != null) {
try {
getUnderlyingOutputStream().close();
} finally {
if (tocWriter != null) {
tocWriter.close();
}
}
}
}
} catch (final IOException ioe) {
markDirty();
throw ioe;
} finally {
unlock();
}
}
@Override
public int getRecordsWritten() {
return recordsWritten;
}
@Override
public File getFile() {
return file;
}
@Override
public void lock() {
lock.lock();
}
@Override
public void unlock() {
lock.unlock();
}
@Override
public boolean tryLock() {
final boolean obtainedLock = lock.tryLock();
if (obtainedLock && isDirty()) {
// once we have obtained the lock, we need to check if the writer
// has been marked dirty. If so, we cannot write to the underlying
// file, so we need to unlock and return false. Otherwise, it's okay
// to write to the underlying file, so return true.
lock.unlock();
return false;
}
return obtainedLock;
}
@Override
public void markDirty() {
this.dirty = true;
}
public boolean isDirty() {
return dirty;
}
protected void resetDirtyFlag() {
this.dirty = false;
}
@Override
public void sync() throws IOException {
try {
if (tocWriter != null) {
tocWriter.sync();
}
syncUnderlyingOutputStream();
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
@Override
public TocWriter getTocWriter() {
return tocWriter;
}
@Override
public boolean isClosed() {
return closed;
}
protected abstract OutputStream getBufferedOutputStream();
protected abstract OutputStream getUnderlyingOutputStream();
protected abstract void syncUnderlyingOutputStream() throws IOException;
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.provenance.schema.EventRecord;
import org.apache.nifi.provenance.serialization.CompressableRecordReader;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.SchemaRecordReader;
import org.apache.nifi.stream.io.LimitingInputStream;
import org.apache.nifi.stream.io.StreamUtils;
public class ByteArraySchemaRecordReader extends CompressableRecordReader {
private RecordSchema schema; // effectively final
private SchemaRecordReader recordReader; // effectively final
public ByteArraySchemaRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException {
super(in, filename, maxAttributeChars);
}
public ByteArraySchemaRecordReader(final InputStream in, final String filename, final TocReader tocReader, final int maxAttributeChars) throws IOException {
super(in, filename, tocReader, maxAttributeChars);
}
private void verifySerializationVersion(final int serializationVersion) {
if (serializationVersion > ByteArraySchemaRecordWriter.SERIALIZATION_VERSION) {
throw new IllegalArgumentException("Unable to deserialize record because the version is " + serializationVersion
+ " and supported versions are 1-" + ByteArraySchemaRecordWriter.SERIALIZATION_VERSION);
}
}
@Override
protected void readHeader(final DataInputStream in, final int serializationVersion) throws IOException {
verifySerializationVersion(serializationVersion);
final int schemaLength = in.readInt();
final byte[] buffer = new byte[schemaLength];
StreamUtils.fillBuffer(in, buffer);
try (final ByteArrayInputStream bais = new ByteArrayInputStream(buffer)) {
schema = RecordSchema.readFrom(bais);
}
recordReader = SchemaRecordReader.fromSchema(schema);
}
@Override
protected StandardProvenanceEventRecord nextRecord(final DataInputStream in, final int serializationVersion) throws IOException {
verifySerializationVersion(serializationVersion);
final long byteOffset = getBytesConsumed();
final int recordLength = in.readInt();
final InputStream limitedIn = new LimitingInputStream(in, recordLength);
final Record eventRecord = recordReader.readRecord(limitedIn);
return EventRecord.getEvent(eventRecord, getFilename(), byteOffset, getMaxAttributeLength());
}
}

View File

@ -0,0 +1,85 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.nifi.provenance.schema.EventRecord;
import org.apache.nifi.provenance.schema.EventRecordFields;
import org.apache.nifi.provenance.schema.ProvenanceEventSchema;
import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.SchemaRecordWriter;
import org.apache.nifi.stream.io.DataOutputStream;
public class ByteArraySchemaRecordWriter extends CompressableRecordWriter {
private static final RecordSchema eventSchema = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1;
private static final RecordSchema contentClaimSchema = new RecordSchema(eventSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields());
public static final int SERIALIZATION_VERSION = 1;
public static final String SERIALIZATION_NAME = "ByteArraySchemaRecordWriter";
private final SchemaRecordWriter recordWriter = new SchemaRecordWriter();
public ByteArraySchemaRecordWriter(final File file, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(file, tocWriter, compressed, uncompressedBlockSize);
}
public ByteArraySchemaRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(out, tocWriter, compressed, uncompressedBlockSize);
}
@Override
protected String getSerializationName() {
return SERIALIZATION_NAME;
}
@Override
protected int getSerializationVersion() {
return SERIALIZATION_VERSION;
}
@Override
public void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
eventSchema.writeTo(baos);
out.writeInt(baos.size());
baos.writeTo(out);
}
protected Record createRecord(final ProvenanceEventRecord event, final long eventId) {
return new EventRecord(event, eventId, eventSchema, contentClaimSchema);
}
@Override
protected void writeRecord(final ProvenanceEventRecord event, final long eventId, final DataOutputStream out) throws IOException {
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(256)) {
final Record eventRecord = createRecord(event, eventId);
recordWriter.writeRecord(eventRecord, baos);
out.writeInt(baos.size());
baos.writeTo(out);
}
}
}

View File

@ -110,6 +110,11 @@ import org.apache.nifi.util.RingBuffer;
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
import org.apache.nifi.util.StopWatch;
import org.apache.nifi.util.Tuple;
import org.apache.nifi.util.timebuffer.CountSizeEntityAccess;
import org.apache.nifi.util.timebuffer.LongEntityAccess;
import org.apache.nifi.util.timebuffer.TimedBuffer;
import org.apache.nifi.util.timebuffer.TimedCountSize;
import org.apache.nifi.util.timebuffer.TimestampedLong;
import org.apache.nifi.web.ResourceNotFoundException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -120,7 +125,6 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
private static final String FILE_EXTENSION = ".prov";
private static final String TEMP_FILE_SUFFIX = ".prov.part";
private static final long PURGE_EVENT_MILLISECONDS = 2500L; //Determines the frequency over which the task to delete old events will occur
public static final int SERIALIZATION_VERSION = 9;
public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
public static final Pattern INDEX_PATTERN = Pattern.compile("index-\\d+");
public static final Pattern LOG_FILENAME_PATTERN = Pattern.compile("(\\d+).*\\.prov");
@ -180,6 +184,9 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
private Authorizer authorizer; // effectively final
private ProvenanceAuthorizableFactory resourceFactory; // effectively final
private final TimedBuffer<TimedCountSize> updateCounts = new TimedBuffer<>(TimeUnit.SECONDS, 300, new CountSizeEntityAccess());
private final TimedBuffer<TimestampedLong> backpressurePauseMillis = new TimedBuffer<>(TimeUnit.SECONDS, 300, new LongEntityAccess());
/**
* default no args constructor for service loading only.
*/
@ -401,7 +408,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
final File journalDirectory = new File(storageDirectory, "journals");
final File journalFile = new File(journalDirectory, String.valueOf(initialRecordId) + ".journal." + i);
writers[i] = RecordWriters.newRecordWriter(journalFile, false, false);
writers[i] = RecordWriters.newSchemaRecordWriter(journalFile, false, false);
writers[i].writeHeader(initialRecordId);
}
@ -762,18 +769,23 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
try {
try {
long recordsWritten = 0L;
for (final ProvenanceEventRecord nextRecord : records) {
final long eventId = idGenerator.getAndIncrement();
bytesWritten += writer.writeRecord(nextRecord, eventId);
recordsWritten++;
logger.trace("Wrote record with ID {} to {}", eventId, writer);
}
writer.flush();
if (alwaysSync) {
writer.sync();
}
totalJournalSize = bytesWrittenSinceRollover.addAndGet(bytesWritten);
recordsWrittenSinceRollover.getAndIncrement();
this.updateCounts.add(new TimedCountSize(recordsWritten, bytesWritten));
} catch (final Throwable t) {
// We need to set the repoDirty flag before we release the lock for this journal.
// Otherwise, another thread may write to this journal -- this is a problem because
@ -1331,14 +1343,17 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
updated = idToPathMap.compareAndSet(existingPathMap, newIdToPathMap);
}
logger.info("Successfully Rolled over Provenance Event file containing {} records", recordsWritten);
final TimedCountSize countSize = updateCounts.getAggregateValue(System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES));
logger.info("Successfully Rolled over Provenance Event file containing {} records. In the past 5 minutes, "
+ "{} events have been written to the Provenance Repository, totaling {}",
recordsWritten, countSize.getCount(), FormatUtils.formatDataSize(countSize.getSize()));
}
//if files were rolled over or if out of retries stop the future
if (fileRolledOver != null || retryAttempts.decrementAndGet() == 0) {
if (fileRolledOver == null && retryAttempts.get() == 0) {
logger.error("Failed to merge Journal Files {} after {} attempts. ", journalsToMerge, MAX_JOURNAL_ROLLOVER_RETRIES);
logger.error("Failed to merge Journal Files {} after {} attempts.", journalsToMerge, MAX_JOURNAL_ROLLOVER_RETRIES);
}
rolloverCompletions.getAndIncrement();
@ -1387,6 +1402,8 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
// max capacity for the repo, or if we have 5 sets of journal files waiting to be merged, we will block here until
// that is no longer the case.
if (journalFileCount > journalCountThreshold || repoSize > sizeThreshold) {
final long stopTheWorldStart = System.nanoTime();
logger.warn("The rate of the dataflow is exceeding the provenance recording rate. "
+ "Slowing down flow to accommodate. Currently, there are {} journal files ({} bytes) and "
+ "threshold for blocking is {} ({} bytes)", journalFileCount, repoSize, journalCountThreshold, sizeThreshold);
@ -1428,8 +1445,12 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
repoSize = getSize(getLogFiles(), 0L);
}
final long stopTheWorldNanos = System.nanoTime() - stopTheWorldStart;
backpressurePauseMillis.add(new TimestampedLong(stopTheWorldNanos));
final TimestampedLong pauseNanosLastFiveMinutes = backpressurePauseMillis.getAggregateValue(System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES));
logger.info("Provenance Repository has now caught up with rolling over journal files. Current number of "
+ "journal files to be rolled over is {}", journalFileCount);
+ "journal files to be rolled over is {}. Provenance Repository Back Pressure paused Session commits for {} ({} total in the last 5 minutes).",
journalFileCount, FormatUtils.formatNanos(stopTheWorldNanos, true), FormatUtils.formatNanos(pauseNanosLastFiveMinutes.getValue(), true));
}
// we've finished rolling over successfully. Create new writers and reset state.
@ -1635,7 +1656,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
}
if (eventReporter != null) {
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "re " + ioe.toString());
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to merge Journal Files due to " + ioe.toString());
}
}
}
@ -1696,7 +1717,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
// loop over each entry in the map, persisting the records to the merged file in order, and populating the map
// with the next entry from the journal file from which the previous record was written.
try (final RecordWriter writer = RecordWriters.newRecordWriter(writerFile, configuration.isCompressOnRollover(), true)) {
try (final RecordWriter writer = RecordWriters.newSchemaRecordWriter(writerFile, configuration.isCompressOnRollover(), true)) {
writer.writeHeader(minEventId);
final IndexingAction indexingAction = createIndexingAction();
@ -1903,13 +1924,22 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
private StandardProvenanceEventRecord truncateAttributes(final StandardProvenanceEventRecord original) {
boolean requireTruncation = false;
for (final Map.Entry<String, String> entry : original.getAttributes().entrySet()) {
if (entry.getValue().length() > maxAttributeChars) {
for (final String updatedAttr : original.getUpdatedAttributes().values()) {
if (updatedAttr != null && updatedAttr.length() > maxAttributeChars) {
requireTruncation = true;
break;
}
}
if (!requireTruncation) {
for (final String previousAttr : original.getPreviousAttributes().values()) {
if (previousAttr != null && previousAttr.length() > maxAttributeChars) {
requireTruncation = true;
break;
}
}
}
if (!requireTruncation) {
return original;
}

View File

@ -17,178 +17,36 @@
package org.apache.nifi.provenance;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.regex.Pattern;
import java.util.zip.GZIPInputStream;
import org.apache.nifi.provenance.serialization.RecordReader;
import org.apache.nifi.provenance.serialization.CompressableRecordReader;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.ByteCountingInputStream;
import org.apache.nifi.stream.io.LimitingInputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardRecordReader implements RecordReader {
public class StandardRecordReader extends CompressableRecordReader {
private static final Logger logger = LoggerFactory.getLogger(StandardRecordReader.class);
private static final Pattern UUID_PATTERN = Pattern.compile("[a-fA-F0-9]{8}\\-([a-fA-F0-9]{4}\\-){3}[a-fA-F0-9]{12}");
private final ByteCountingInputStream rawInputStream;
private final String filename;
private final int serializationVersion;
private final boolean compressed;
private final TocReader tocReader;
private final int headerLength;
private final int maxAttributeChars;
private DataInputStream dis;
private ByteCountingInputStream byteCountingIn;
public StandardRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException {
this(in, filename, null, maxAttributeChars);
}
public StandardRecordReader(final InputStream in, final String filename, final TocReader tocReader, final int maxAttributeChars) throws IOException {
super(in, filename, tocReader, maxAttributeChars);
logger.trace("Creating RecordReader for {}", filename);
rawInputStream = new ByteCountingInputStream(in);
this.maxAttributeChars = maxAttributeChars;
final InputStream limitedStream;
if ( tocReader == null ) {
limitedStream = rawInputStream;
} else {
final long offset1 = tocReader.getBlockOffset(1);
if ( offset1 < 0 ) {
limitedStream = rawInputStream;
} else {
limitedStream = new LimitingInputStream(rawInputStream, offset1 - rawInputStream.getBytesConsumed());
}
}
final InputStream readableStream;
if (filename.endsWith(".gz")) {
readableStream = new BufferedInputStream(new GZIPInputStream(limitedStream));
compressed = true;
} else {
readableStream = new BufferedInputStream(limitedStream);
compressed = false;
}
byteCountingIn = new ByteCountingInputStream(readableStream);
dis = new DataInputStream(byteCountingIn);
final String repoClassName = dis.readUTF();
final int serializationVersion = dis.readInt();
headerLength = repoClassName.getBytes(StandardCharsets.UTF_8).length + 2 + 4; // 2 bytes for string length, 4 for integer.
if (serializationVersion < 1 || serializationVersion > 9) {
throw new IllegalArgumentException("Unable to deserialize record because the version is " + serializationVersion + " and supported versions are 1-9");
}
this.serializationVersion = serializationVersion;
this.filename = filename;
this.tocReader = tocReader;
}
@Override
public void skipToBlock(final int blockIndex) throws IOException {
if ( tocReader == null ) {
throw new IllegalStateException("Cannot skip to block " + blockIndex + " for Provenance Log " + filename + " because no Table-of-Contents file was found for this Log");
}
if ( blockIndex < 0 ) {
throw new IllegalArgumentException("Cannot skip to block " + blockIndex + " because the value is negative");
}
if ( blockIndex == getBlockIndex() ) {
return;
}
final long offset = tocReader.getBlockOffset(blockIndex);
if ( offset < 0 ) {
throw new IOException("Unable to find block " + blockIndex + " in Provenance Log " + filename);
}
final long curOffset = rawInputStream.getBytesConsumed();
final long bytesToSkip = offset - curOffset;
if ( bytesToSkip >= 0 ) {
try {
StreamUtils.skip(rawInputStream, bytesToSkip);
logger.debug("Skipped stream from offset {} to {} ({} bytes skipped)", curOffset, offset, bytesToSkip);
} catch (final IOException e) {
throw new IOException("Failed to skip to offset " + offset + " for block " + blockIndex + " of Provenance Log " + filename, e);
}
resetStreamForNextBlock();
}
}
private void resetStreamForNextBlock() throws IOException {
final InputStream limitedStream;
if ( tocReader == null ) {
limitedStream = rawInputStream;
} else {
final long offset = tocReader.getBlockOffset(1 + getBlockIndex());
if ( offset < 0 ) {
limitedStream = rawInputStream;
} else {
limitedStream = new LimitingInputStream(rawInputStream, offset - rawInputStream.getBytesConsumed());
}
}
final InputStream readableStream;
if (compressed) {
readableStream = new BufferedInputStream(new GZIPInputStream(limitedStream));
} else {
readableStream = new BufferedInputStream(limitedStream);
}
byteCountingIn = new ByteCountingInputStream(readableStream, rawInputStream.getBytesConsumed());
dis = new DataInputStream(byteCountingIn);
}
@Override
public TocReader getTocReader() {
return tocReader;
}
@Override
public boolean isBlockIndexAvailable() {
return tocReader != null;
}
@Override
public int getBlockIndex() {
if ( tocReader == null ) {
throw new IllegalStateException("Cannot determine Block Index because no Table-of-Contents could be found for Provenance Log " + filename);
}
return tocReader.getBlockIndex(rawInputStream.getBytesConsumed());
}
@Override
public long getBytesConsumed() {
return byteCountingIn.getBytesConsumed();
}
private StandardProvenanceEventRecord readPreVersion6Record() throws IOException {
final long startOffset = byteCountingIn.getBytesConsumed();
if (!isData()) {
return null;
}
private StandardProvenanceEventRecord readPreVersion6Record(final DataInputStream dis, final int serializationVersion) throws IOException {
final long startOffset = getBytesConsumed();
final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
final long eventId = dis.readLong();
@ -254,7 +112,7 @@ public class StandardRecordReader implements RecordReader {
builder.setAttributes(Collections.<String, String>emptyMap(), attrs);
builder.setCurrentContentClaim(null, null, null, null, fileSize);
builder.setStorageLocation(filename, startOffset);
builder.setStorageLocation(getFilename(), startOffset);
final StandardProvenanceEventRecord record = builder.build();
record.setEventId(eventId);
@ -262,17 +120,18 @@ public class StandardRecordReader implements RecordReader {
}
@Override
public StandardProvenanceEventRecord nextRecord() throws IOException {
public StandardProvenanceEventRecord nextRecord(final DataInputStream dis, final int serializationVersion) throws IOException {
if (serializationVersion > StandardRecordWriter.SERIALIZATION_VERISON) {
throw new IllegalArgumentException("Unable to deserialize record because the version is "
+ serializationVersion + " and supported versions are 1-" + StandardRecordWriter.SERIALIZATION_VERISON);
}
// Schema changed drastically in version 6 so we created a new method to handle old records
if (serializationVersion < 6) {
return readPreVersion6Record();
return readPreVersion6Record(dis, serializationVersion);
}
final long startOffset = byteCountingIn.getBytesConsumed();
if (!isData()) {
return null;
}
final long startOffset = getBytesConsumed();
final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
@ -288,7 +147,7 @@ public class StandardRecordReader implements RecordReader {
if (serializationVersion < 9){
final int numLineageIdentifiers = dis.readInt();
for (int i = 0; i < numLineageIdentifiers; i++) {
readUUID(dis); //skip identifiers
readUUID(dis, serializationVersion); //skip identifiers
}
}
@ -303,7 +162,7 @@ public class StandardRecordReader implements RecordReader {
builder.setComponentId(readNullableString(dis));
builder.setComponentType(readNullableString(dis));
final String uuid = readUUID(dis);
final String uuid = readUUID(dis, serializationVersion);
builder.setFlowFileUUID(uuid);
builder.setDetails(readNullableString(dis));
@ -335,12 +194,12 @@ public class StandardRecordReader implements RecordReader {
if (eventType == ProvenanceEventType.FORK || eventType == ProvenanceEventType.JOIN || eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.REPLAY) {
final int numParents = dis.readInt();
for (int i = 0; i < numParents; i++) {
builder.addParentUuid(readUUID(dis));
builder.addParentUuid(readUUID(dis, serializationVersion));
}
final int numChildren = dis.readInt();
for (int i = 0; i < numChildren; i++) {
builder.addChildUuid(readUUID(dis));
builder.addChildUuid(readUUID(dis, serializationVersion));
}
} else if (eventType == ProvenanceEventType.RECEIVE) {
builder.setTransitUri(readNullableString(dis));
@ -357,7 +216,7 @@ public class StandardRecordReader implements RecordReader {
builder.setFlowFileEntryDate(flowFileEntryDate);
builder.setLineageStartDate(lineageStartDate);
builder.setStorageLocation(filename, startOffset);
builder.setStorageLocation(getFilename(), startOffset);
final StandardProvenanceEventRecord record = builder.build();
record.setEventId(eventId);
@ -373,8 +232,8 @@ public class StandardRecordReader implements RecordReader {
final String truncatedValue;
if (value == null) {
truncatedValue = null;
} else if (value.length() > maxAttributeChars) {
truncatedValue = value.substring(0, maxAttributeChars);
} else if (value.length() > getMaxAttributeLength()) {
truncatedValue = value.substring(0, getMaxAttributeLength());
} else {
truncatedValue = value;
}
@ -385,8 +244,8 @@ public class StandardRecordReader implements RecordReader {
return attrs;
}
private String readUUID(final DataInputStream in) throws IOException {
if ( serializationVersion < 8 ) {
private String readUUID(final DataInputStream in, final int serializationVersion) throws IOException {
if (serializationVersion < 8) {
final long msb = in.readLong();
final long lsb = in.readLong();
return new UUID(msb, lsb).toString();
@ -427,80 +286,4 @@ public class StandardRecordReader implements RecordReader {
StreamUtils.fillBuffer(in, strBytes);
return new String(strBytes, "UTF-8");
}
private boolean isData() throws IOException {
byteCountingIn.mark(1);
int nextByte = byteCountingIn.read();
byteCountingIn.reset();
if ( nextByte < 0 ) {
try {
resetStreamForNextBlock();
} catch (final EOFException eof) {
return false;
}
byteCountingIn.mark(1);
nextByte = byteCountingIn.read();
byteCountingIn.reset();
}
return nextByte >= 0;
}
@Override
public long getMaxEventId() throws IOException {
if ( tocReader != null ) {
final long lastBlockOffset = tocReader.getLastBlockOffset();
skipToBlock(tocReader.getBlockIndex(lastBlockOffset));
}
ProvenanceEventRecord record;
ProvenanceEventRecord lastRecord = null;
try {
while ((record = nextRecord()) != null) {
lastRecord = record;
}
} catch (final EOFException eof) {
// This can happen if we stop NIFi while the record is being written.
// This is OK, we just ignore this record. The session will not have been
// committed, so we can just process the FlowFile again.
}
return lastRecord == null ? -1L : lastRecord.getEventId();
}
@Override
public void close() throws IOException {
logger.trace("Closing Record Reader for {}", filename);
dis.close();
rawInputStream.close();
if ( tocReader != null ) {
tocReader.close();
}
}
@Override
public void skip(final long bytesToSkip) throws IOException {
StreamUtils.skip(dis, bytesToSkip);
}
@Override
public void skipTo(final long position) throws IOException {
// we are subtracting headerLength from the number of bytes consumed because we used to
// consider the offset of the first record "0" - now we consider it whatever position it
// it really is in the stream.
final long currentPosition = byteCountingIn.getBytesConsumed() - headerLength;
if (currentPosition == position) {
return;
}
if (currentPosition > position) {
throw new IOException("Cannot skip to byte offset " + position + " in stream because already at byte offset " + currentPosition);
}
final long toSkip = position - currentPosition;
StreamUtils.skip(dis, toSkip);
}
}

View File

@ -17,239 +17,142 @@
package org.apache.nifi.provenance;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.ByteCountingOutputStream;
import org.apache.nifi.stream.io.DataOutputStream;
import org.apache.nifi.stream.io.GZIPOutputStream;
import org.apache.nifi.stream.io.NonCloseableOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardRecordWriter implements RecordWriter {
/**
* @deprecated Deprecated in favor of SchemaRecordWriter
*/
@Deprecated
public class StandardRecordWriter extends CompressableRecordWriter implements RecordWriter {
private static final Logger logger = LoggerFactory.getLogger(StandardRecordWriter.class);
public static final int SERIALIZATION_VERISON = 9;
public static final String SERIALIZATION_NAME = "org.apache.nifi.provenance.PersistentProvenanceRepository";
private final File file;
private final FileOutputStream fos;
private final ByteCountingOutputStream rawOutStream;
private final TocWriter tocWriter;
private final boolean compressed;
private final int uncompressedBlockSize;
private final AtomicBoolean dirtyFlag = new AtomicBoolean(false);
private DataOutputStream out;
private ByteCountingOutputStream byteCountingOut;
private long lastBlockOffset = 0L;
private int recordCount = 0;
private volatile boolean closed = false;
private final Lock lock = new ReentrantLock();
public StandardRecordWriter(final File file, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(file, writer, compressed, uncompressedBlockSize);
logger.trace("Creating Record Writer for {}", file.getName());
this.file = file;
this.compressed = compressed;
this.fos = new FileOutputStream(file);
rawOutStream = new ByteCountingOutputStream(fos);
this.uncompressedBlockSize = uncompressedBlockSize;
}
this.tocWriter = writer;
public StandardRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(out, tocWriter, compressed, uncompressedBlockSize);
this.file = null;
}
@Override
public synchronized File getFile() {
return file;
protected String getSerializationName() {
return SERIALIZATION_NAME;
}
@Override
public synchronized void writeHeader(final long firstEventId) throws IOException {
if (isDirty()) {
throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository");
}
try {
lastBlockOffset = rawOutStream.getBytesWritten();
resetWriteStream(firstEventId);
out.writeUTF(PersistentProvenanceRepository.class.getName());
out.writeInt(PersistentProvenanceRepository.SERIALIZATION_VERSION);
out.flush();
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
/**
* Resets the streams to prepare for a new block
* @param eventId the first id that will be written to the new block
* @throws IOException if unable to flush/close the current streams properly
*/
private void resetWriteStream(final long eventId) throws IOException {
try {
if (out != null) {
out.flush();
}
final long byteOffset = (byteCountingOut == null) ? rawOutStream.getBytesWritten() : byteCountingOut.getBytesWritten();
final OutputStream writableStream;
if ( compressed ) {
// because of the way that GZIPOutputStream works, we need to call close() on it in order for it
// to write its trailing bytes. But we don't want to close the underlying OutputStream, so we wrap
// the underlying OutputStream in a NonCloseableOutputStream
// We don't have to check if the writer is dirty because we will have already checked before calling this method.
if ( out != null ) {
out.close();
}
if ( tocWriter != null ) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId);
}
writableStream = new BufferedOutputStream(new GZIPOutputStream(new NonCloseableOutputStream(rawOutStream), 1), 65536);
} else {
if ( tocWriter != null ) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId);
}
writableStream = new BufferedOutputStream(rawOutStream, 65536);
}
this.byteCountingOut = new ByteCountingOutputStream(writableStream, byteOffset);
this.out = new DataOutputStream(byteCountingOut);
dirtyFlag.set(false);
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
protected int getSerializationVersion() {
return SERIALIZATION_VERISON;
}
@Override
public synchronized long writeRecord(final ProvenanceEventRecord record, final long recordIdentifier) throws IOException {
if (isDirty()) {
throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository");
protected void writeHeader(long firstEventId, DataOutputStream out) throws IOException {
}
@Override
protected void writeRecord(final ProvenanceEventRecord record, final long recordIdentifier, final DataOutputStream out) throws IOException {
final ProvenanceEventType recordType = record.getEventType();
out.writeLong(recordIdentifier);
out.writeUTF(record.getEventType().name());
out.writeLong(record.getEventTime());
out.writeLong(record.getFlowFileEntryDate());
out.writeLong(record.getEventDuration());
out.writeLong(record.getLineageStartDate());
writeNullableString(out, record.getComponentId());
writeNullableString(out, record.getComponentType());
writeUUID(out, record.getFlowFileUuid());
writeNullableString(out, record.getDetails());
// Write FlowFile attributes
final Map<String, String> attrs = record.getPreviousAttributes();
out.writeInt(attrs.size());
for (final Map.Entry<String, String> entry : attrs.entrySet()) {
writeLongString(out, entry.getKey());
writeLongString(out, entry.getValue());
}
try {
final ProvenanceEventType recordType = record.getEventType();
final long startBytes = byteCountingOut.getBytesWritten();
final Map<String, String> attrUpdates = record.getUpdatedAttributes();
out.writeInt(attrUpdates.size());
for (final Map.Entry<String, String> entry : attrUpdates.entrySet()) {
writeLongString(out, entry.getKey());
writeLongNullableString(out, entry.getValue());
}
// add a new block to the TOC if needed.
if ( tocWriter != null && (startBytes - lastBlockOffset >= uncompressedBlockSize) ) {
lastBlockOffset = startBytes;
if ( compressed ) {
// because of the way that GZIPOutputStream works, we need to call close() on it in order for it
// to write its trailing bytes. But we don't want to close the underlying OutputStream, so we wrap
// the underlying OutputStream in a NonCloseableOutputStream
resetWriteStream(recordIdentifier);
}
}
out.writeLong(recordIdentifier);
out.writeUTF(record.getEventType().name());
out.writeLong(record.getEventTime());
out.writeLong(record.getFlowFileEntryDate());
out.writeLong(record.getEventDuration());
out.writeLong(record.getLineageStartDate());
writeNullableString(out, record.getComponentId());
writeNullableString(out, record.getComponentType());
writeUUID(out, record.getFlowFileUuid());
writeNullableString(out, record.getDetails());
// Write FlowFile attributes
final Map<String, String> attrs = record.getPreviousAttributes();
out.writeInt(attrs.size());
for (final Map.Entry<String, String> entry : attrs.entrySet()) {
writeLongString(out, entry.getKey());
writeLongString(out, entry.getValue());
}
final Map<String, String> attrUpdates = record.getUpdatedAttributes();
out.writeInt(attrUpdates.size());
for (final Map.Entry<String, String> entry : attrUpdates.entrySet()) {
writeLongString(out, entry.getKey());
writeLongNullableString(out, entry.getValue());
}
// If Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
if (record.getContentClaimSection() != null && record.getContentClaimContainer() != null && record.getContentClaimIdentifier() != null) {
out.writeBoolean(true);
out.writeUTF(record.getContentClaimContainer());
out.writeUTF(record.getContentClaimSection());
out.writeUTF(record.getContentClaimIdentifier());
if (record.getContentClaimOffset() == null) {
out.writeLong(0L);
} else {
out.writeLong(record.getContentClaimOffset());
}
out.writeLong(record.getFileSize());
// If Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
if (record.getContentClaimSection() != null && record.getContentClaimContainer() != null && record.getContentClaimIdentifier() != null) {
out.writeBoolean(true);
out.writeUTF(record.getContentClaimContainer());
out.writeUTF(record.getContentClaimSection());
out.writeUTF(record.getContentClaimIdentifier());
if (record.getContentClaimOffset() == null) {
out.writeLong(0L);
} else {
out.writeBoolean(false);
out.writeLong(record.getContentClaimOffset());
}
out.writeLong(record.getFileSize());
} else {
out.writeBoolean(false);
}
// If Previous Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
if (record.getPreviousContentClaimSection() != null && record.getPreviousContentClaimContainer() != null && record.getPreviousContentClaimIdentifier() != null) {
out.writeBoolean(true);
out.writeUTF(record.getPreviousContentClaimContainer());
out.writeUTF(record.getPreviousContentClaimSection());
out.writeUTF(record.getPreviousContentClaimIdentifier());
if (record.getPreviousContentClaimOffset() == null) {
out.writeLong(0L);
} else {
out.writeLong(record.getPreviousContentClaimOffset());
}
if (record.getPreviousFileSize() == null) {
out.writeLong(0L);
} else {
out.writeLong(record.getPreviousFileSize());
}
// If Previous Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
if (record.getPreviousContentClaimSection() != null && record.getPreviousContentClaimContainer() != null && record.getPreviousContentClaimIdentifier() != null) {
out.writeBoolean(true);
out.writeUTF(record.getPreviousContentClaimContainer());
out.writeUTF(record.getPreviousContentClaimSection());
out.writeUTF(record.getPreviousContentClaimIdentifier());
if (record.getPreviousContentClaimOffset() == null) {
out.writeLong(0L);
} else {
out.writeBoolean(false);
out.writeLong(record.getPreviousContentClaimOffset());
}
// write out the identifier of the destination queue.
writeNullableString(out, record.getSourceQueueIdentifier());
// Write type-specific info
if (recordType == ProvenanceEventType.FORK || recordType == ProvenanceEventType.JOIN || recordType == ProvenanceEventType.CLONE || recordType == ProvenanceEventType.REPLAY) {
writeUUIDs(out, record.getParentUuids());
writeUUIDs(out, record.getChildUuids());
} else if (recordType == ProvenanceEventType.RECEIVE) {
writeNullableString(out, record.getTransitUri());
writeNullableString(out, record.getSourceSystemFlowFileIdentifier());
} else if (recordType == ProvenanceEventType.FETCH) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.SEND) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.ADDINFO) {
writeNullableString(out, record.getAlternateIdentifierUri());
} else if (recordType == ProvenanceEventType.ROUTE) {
writeNullableString(out, record.getRelationship());
if (record.getPreviousFileSize() == null) {
out.writeLong(0L);
} else {
out.writeLong(record.getPreviousFileSize());
}
} else {
out.writeBoolean(false);
}
out.flush();
recordCount++;
return byteCountingOut.getBytesWritten() - startBytes;
} catch (final IOException ioe) {
markDirty();
throw ioe;
// write out the identifier of the destination queue.
writeNullableString(out, record.getSourceQueueIdentifier());
// Write type-specific info
if (recordType == ProvenanceEventType.FORK || recordType == ProvenanceEventType.JOIN || recordType == ProvenanceEventType.CLONE || recordType == ProvenanceEventType.REPLAY) {
writeUUIDs(out, record.getParentUuids());
writeUUIDs(out, record.getChildUuids());
} else if (recordType == ProvenanceEventType.RECEIVE) {
writeNullableString(out, record.getTransitUri());
writeNullableString(out, record.getSourceSystemFlowFileIdentifier());
} else if (recordType == ProvenanceEventType.FETCH) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.SEND) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.ADDINFO) {
writeNullableString(out, record.getAlternateIdentifierUri());
} else if (recordType == ProvenanceEventType.ROUTE) {
writeNullableString(out, record.getRelationship());
}
}
@ -292,110 +195,9 @@ public class StandardRecordWriter implements RecordWriter {
out.write(bytes);
}
@Override
public synchronized void close() throws IOException {
closed = true;
logger.trace("Closing Record Writer for {}", file.getName());
lock();
try {
try {
// We want to close 'out' only if the writer is not 'dirty'.
// If the writer is dirty, then there was a failure to write
// to disk, which means that we likely have a partial record written
// to disk.
//
// If we call close() on out, it will in turn call flush() on the underlying
// output stream, which is a BufferedOutputStream. As a result, we will end
// up flushing the buffer after a partially written record, which results in
// essentially random bytes being written to the repository, which causes
// corruption and un-recoverability. Since we will close the underlying 'rawOutStream'
// below, we will still appropriately clean up the resources help by this writer, so
// we are still OK in terms of closing all resources held by the writer.
if (out != null && !isDirty()) {
out.close();
}
} finally {
try {
rawOutStream.close();
} finally {
if (tocWriter != null) {
tocWriter.close();
}
}
}
} catch (final IOException ioe) {
markDirty();
throw ioe;
} finally {
unlock();
}
}
@Override
public boolean isClosed() {
return closed;
}
@Override
public synchronized int getRecordsWritten() {
return recordCount;
}
@Override
public void lock() {
lock.lock();
}
@Override
public void unlock() {
lock.unlock();
}
@Override
public boolean tryLock() {
final boolean obtainedLock = lock.tryLock();
if (obtainedLock && dirtyFlag.get()) {
// once we have obtained the lock, we need to check if the writer
// has been marked dirty. If so, we cannot write to the underlying
// file, so we need to unlock and return false. Otherwise, it's okay
// to write to the underlying file, so return true.
lock.unlock();
return false;
}
return obtainedLock;
}
@Override
public String toString() {
return "StandardRecordWriter[file=" + file + "]";
}
@Override
public void sync() throws IOException {
try {
if ( tocWriter != null ) {
tocWriter.sync();
}
fos.getFD().sync();
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
@Override
public TocWriter getTocWriter() {
return tocWriter;
}
@Override
public void markDirty() {
dirtyFlag.set(true);
}
public boolean isDirty() {
return dirtyFlag.get();
}
}

View File

@ -19,7 +19,6 @@ package org.apache.nifi.provenance.lucene;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.document.Document;
@ -45,7 +44,7 @@ public class IndexingAction {
}
private void addField(final Document doc, final SearchableField field, final String value, final Store store) {
if (value == null || (!nonAttributeSearchableFields.contains(field) && !field.isAttribute())) {
if (value == null || (!field.isAttribute() && !nonAttributeSearchableFields.contains(field))) {
return;
}
@ -54,11 +53,9 @@ public class IndexingAction {
public void index(final StandardProvenanceEventRecord record, final IndexWriter indexWriter, final Integer blockIndex) throws IOException {
final Map<String, String> attributes = record.getAttributes();
final Document doc = new Document();
addField(doc, SearchableFields.FlowFileUUID, record.getFlowFileUuid(), Store.NO);
addField(doc, SearchableFields.Filename, attributes.get(CoreAttributes.FILENAME.key()), Store.NO);
addField(doc, SearchableFields.Filename, record.getAttribute(CoreAttributes.FILENAME.key()), Store.NO);
addField(doc, SearchableFields.ComponentID, record.getComponentId(), Store.NO);
addField(doc, SearchableFields.AlternateIdentifierURI, record.getAlternateIdentifierUri(), Store.NO);
addField(doc, SearchableFields.EventType, record.getEventType().name(), Store.NO);
@ -68,13 +65,10 @@ public class IndexingAction {
addField(doc, SearchableFields.ContentClaimContainer, record.getContentClaimContainer(), Store.NO);
addField(doc, SearchableFields.ContentClaimIdentifier, record.getContentClaimIdentifier(), Store.NO);
addField(doc, SearchableFields.SourceQueueIdentifier, record.getSourceQueueIdentifier(), Store.NO);
if (nonAttributeSearchableFields.contains(SearchableFields.TransitURI)) {
addField(doc, SearchableFields.TransitURI, record.getTransitUri(), Store.NO);
}
addField(doc, SearchableFields.TransitURI, record.getTransitUri(), Store.NO);
for (final SearchableField searchableField : attributeSearchableFields) {
addField(doc, searchableField, LuceneUtil.truncateIndexField(attributes.get(searchableField.getSearchableFieldName())), Store.NO);
addField(doc, searchableField, LuceneUtil.truncateIndexField(record.getAttribute(searchableField.getSearchableFieldName())), Store.NO);
}
final String storageFilename = LuceneUtil.substringBefore(record.getStorageFilename(), ".");
@ -94,19 +88,20 @@ public class IndexingAction {
}
// If it's event is a FORK, or JOIN, add the FlowFileUUID for all child/parent UUIDs.
if (record.getEventType() == ProvenanceEventType.FORK || record.getEventType() == ProvenanceEventType.CLONE || record.getEventType() == ProvenanceEventType.REPLAY) {
final ProvenanceEventType eventType = record.getEventType();
if (eventType == ProvenanceEventType.FORK || eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.REPLAY) {
for (final String uuid : record.getChildUuids()) {
if (!uuid.equals(record.getFlowFileUuid())) {
addField(doc, SearchableFields.FlowFileUUID, uuid, Store.NO);
}
}
} else if (record.getEventType() == ProvenanceEventType.JOIN) {
} else if (eventType == ProvenanceEventType.JOIN) {
for (final String uuid : record.getParentUuids()) {
if (!uuid.equals(record.getFlowFileUuid())) {
addField(doc, SearchableFields.FlowFileUUID, uuid, Store.NO);
}
}
} else if (record.getEventType() == ProvenanceEventType.RECEIVE && record.getSourceSystemFlowFileIdentifier() != null) {
} else if (eventType == ProvenanceEventType.RECEIVE && record.getSourceSystemFlowFileIdentifier() != null) {
// If we get a receive with a Source System FlowFile Identifier, we add another Document that shows the UUID
// that the Source System uses to refer to the data.
final String sourceIdentifier = record.getSourceSystemFlowFileIdentifier();

View File

@ -0,0 +1,198 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.schema;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.repository.schema.FieldMapRecord;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
public class EventRecord implements Record {
private final RecordSchema schema;
private final ProvenanceEventRecord event;
private final long eventId;
private final Record contentClaimRecord;
private final Record previousClaimRecord;
public EventRecord(final ProvenanceEventRecord event, final long eventId, final RecordSchema schema, final RecordSchema contentClaimSchema) {
this.schema = schema;
this.event = event;
this.eventId = eventId;
this.contentClaimRecord = createContentClaimRecord(contentClaimSchema, event.getContentClaimContainer(), event.getContentClaimSection(),
event.getContentClaimIdentifier(), event.getContentClaimOffset(), event.getFileSize());
this.previousClaimRecord = createContentClaimRecord(contentClaimSchema, event.getPreviousContentClaimContainer(), event.getPreviousContentClaimSection(),
event.getPreviousContentClaimIdentifier(), event.getPreviousContentClaimOffset(), event.getPreviousFileSize());
}
@Override
public RecordSchema getSchema() {
return schema;
}
private static Record createContentClaimRecord(final RecordSchema contentClaimSchema, final String container, final String section,
final String identifier, final Long offset, final Long size) {
if (container == null || section == null || identifier == null) {
return null;
}
final Map<RecordField, Object> fieldValues = new HashMap<>();
fieldValues.put(EventRecordFields.CONTENT_CLAIM_CONTAINER, container);
fieldValues.put(EventRecordFields.CONTENT_CLAIM_SECTION, section);
fieldValues.put(EventRecordFields.CONTENT_CLAIM_IDENTIFIER, identifier);
fieldValues.put(EventRecordFields.CONTENT_CLAIM_OFFSET, offset);
fieldValues.put(EventRecordFields.CONTENT_CLAIM_SIZE, size);
return new FieldMapRecord(fieldValues, contentClaimSchema);
}
@Override
public Object getFieldValue(final String fieldName) {
switch (fieldName) {
case EventRecordFields.Names.EVENT_IDENTIFIER:
return eventId;
case EventRecordFields.Names.ALTERNATE_IDENTIFIER:
return event.getAlternateIdentifierUri();
case EventRecordFields.Names.CHILD_UUIDS:
return event.getChildUuids();
case EventRecordFields.Names.COMPONENT_ID:
return event.getComponentId();
case EventRecordFields.Names.COMPONENT_TYPE:
return event.getComponentType();
case EventRecordFields.Names.CONTENT_CLAIM:
return contentClaimRecord;
case EventRecordFields.Names.EVENT_DETAILS:
return event.getDetails();
case EventRecordFields.Names.EVENT_DURATION:
return event.getEventDuration();
case EventRecordFields.Names.EVENT_TIME:
return event.getEventTime();
case EventRecordFields.Names.EVENT_TYPE:
return event.getEventType().name();
case EventRecordFields.Names.FLOWFILE_ENTRY_DATE:
return event.getFlowFileEntryDate();
case EventRecordFields.Names.FLOWFILE_UUID:
return event.getFlowFileUuid();
case EventRecordFields.Names.LINEAGE_START_DATE:
return event.getLineageStartDate();
case EventRecordFields.Names.PARENT_UUIDS:
return event.getParentUuids();
case EventRecordFields.Names.PREVIOUS_ATTRIBUTES:
return event.getPreviousAttributes();
case EventRecordFields.Names.PREVIOUS_CONTENT_CLAIM:
return previousClaimRecord;
case EventRecordFields.Names.RELATIONSHIP:
return event.getRelationship();
case EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER:
return event.getSourceQueueIdentifier();
case EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER:
return event.getSourceSystemFlowFileIdentifier();
case EventRecordFields.Names.TRANSIT_URI:
return event.getTransitUri();
case EventRecordFields.Names.UPDATED_ATTRIBUTES:
return event.getUpdatedAttributes();
}
return null;
}
@SuppressWarnings("unchecked")
public static StandardProvenanceEventRecord getEvent(final Record record, final String storageFilename, final long storageByteOffset, final int maxAttributeLength) {
final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
builder.setAlternateIdentifierUri((String) record.getFieldValue(EventRecordFields.Names.ALTERNATE_IDENTIFIER));
builder.setChildUuids((List<String>) record.getFieldValue(EventRecordFields.Names.CHILD_UUIDS));
builder.setComponentId((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_ID));
builder.setComponentType((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_TYPE));
builder.setDetails((String) record.getFieldValue(EventRecordFields.Names.EVENT_DETAILS));
builder.setEventDuration((Long) record.getFieldValue(EventRecordFields.Names.EVENT_DURATION));
builder.setEventTime((Long) record.getFieldValue(EventRecordFields.Names.EVENT_TIME));
builder.setEventType(ProvenanceEventType.valueOf((String) record.getFieldValue(EventRecordFields.Names.EVENT_TYPE)));
builder.setFlowFileEntryDate((Long) record.getFieldValue(EventRecordFields.Names.FLOWFILE_ENTRY_DATE));
builder.setFlowFileUUID((String) record.getFieldValue(EventRecordFields.Names.FLOWFILE_UUID));
builder.setLineageStartDate((Long) record.getFieldValue(EventRecordFields.Names.LINEAGE_START_DATE));
builder.setParentUuids((List<String>) record.getFieldValue(EventRecordFields.Names.PARENT_UUIDS));
builder.setPreviousAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventRecordFields.Names.PREVIOUS_ATTRIBUTES), maxAttributeLength));
builder.setEventId((Long) record.getFieldValue(EventRecordFields.Names.EVENT_IDENTIFIER));
builder.setRelationship((String) record.getFieldValue(EventRecordFields.Names.RELATIONSHIP));
builder.setSourceQueueIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER));
builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER));
builder.setTransitUri((String) record.getFieldValue(EventRecordFields.Names.TRANSIT_URI));
builder.setUpdatedAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventRecordFields.Names.UPDATED_ATTRIBUTES), maxAttributeLength));
builder.setStorageLocation(storageFilename, storageByteOffset);
final Record currentClaimRecord = (Record) record.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM);
if (currentClaimRecord == null) {
builder.setCurrentContentClaim(null, null, null, null, 0L);
} else {
builder.setCurrentContentClaim(
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER),
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION),
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER),
(Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET),
(Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE));
}
final Record previousClaimRecord = (Record) record.getFieldValue(EventRecordFields.Names.PREVIOUS_CONTENT_CLAIM);
if (previousClaimRecord != null) {
builder.setPreviousContentClaim(
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER),
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION),
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER),
(Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET),
(Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE));
}
return builder.build();
}
private static Map<String, String> truncateAttributes(final Map<String, String> attributes, final int maxAttributeLength) {
if (attributes == null) {
return null;
}
// Check if any attribute value exceeds the attribute length
final boolean anyExceedsLength = attributes.values().stream()
.filter(value -> value != null)
.anyMatch(value -> value.length() > maxAttributeLength);
if (!anyExceedsLength) {
return attributes;
}
final Map<String, String> truncated = new HashMap<>();
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
final String key = entry.getKey();
final String value = entry.getValue();
if (value == null || value.length() <= maxAttributeLength) {
truncated.put(key, value);
continue;
}
truncated.put(key, value.substring(0, maxAttributeLength));
}
return truncated;
}
}

View File

@ -0,0 +1,110 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.schema;
import static org.apache.nifi.repository.schema.Repetition.EXACTLY_ONE;
import static org.apache.nifi.repository.schema.Repetition.ZERO_OR_MORE;
import static org.apache.nifi.repository.schema.Repetition.ZERO_OR_ONE;
import org.apache.nifi.repository.schema.ComplexRecordField;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.MapRecordField;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.SimpleRecordField;
public class EventRecordFields {
public static class Names {
public static final String EVENT_IDENTIFIER = "Event ID";
public static final String EVENT_TYPE = "Event Type";
public static final String EVENT_TIME = "Event Time";
public static final String FLOWFILE_ENTRY_DATE = "FlowFile Entry Date";
public static final String EVENT_DURATION = "Event Duration";
public static final String LINEAGE_START_DATE = "Lineage Start Date";
public static final String COMPONENT_ID = "Component ID";
public static final String COMPONENT_TYPE = "Component Type";
public static final String FLOWFILE_UUID = "FlowFile UUID";
public static final String EVENT_DETAILS = "Event Details";
public static final String SOURCE_QUEUE_IDENTIFIER = "Source Queue Identifier";
public static final String CONTENT_CLAIM = "Content Claim";
public static final String PREVIOUS_CONTENT_CLAIM = "Previous Content Claim";
public static final String PARENT_UUIDS = "Parent UUIDs";
public static final String CHILD_UUIDS = "Child UUIDs";
public static final String ATTRIBUTE_NAME = "Attribute Name";
public static final String ATTRIBUTE_VALUE = "Attribute Value";
public static final String PREVIOUS_ATTRIBUTES = "Previous Attributes";
public static final String UPDATED_ATTRIBUTES = "Updated Attributes";
public static final String CONTENT_CLAIM_CONTAINER = "Content Claim Container";
public static final String CONTENT_CLAIM_SECTION = "Content Claim Section";
public static final String CONTENT_CLAIM_IDENTIFIER = "Content Claim Identifier";
public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset";
public static final String CONTENT_CLAIM_SIZE = "Content Claim Size";
public static final String TRANSIT_URI = "Transit URI";
public static final String SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = "Source System FlowFile Identifier";
public static final String ALTERNATE_IDENTIFIER = "Alternate Identifier";
public static final String RELATIONSHIP = "Relationship";
}
// General Event fields.
public static final RecordField RECORD_IDENTIFIER = new SimpleRecordField(Names.EVENT_IDENTIFIER, FieldType.LONG, EXACTLY_ONE);
public static final RecordField EVENT_TYPE = new SimpleRecordField(Names.EVENT_TYPE, FieldType.STRING, EXACTLY_ONE);
public static final RecordField EVENT_TIME = new SimpleRecordField(Names.EVENT_TIME, FieldType.LONG, EXACTLY_ONE);
public static final RecordField FLOWFILE_ENTRY_DATE = new SimpleRecordField(Names.FLOWFILE_ENTRY_DATE, FieldType.LONG, EXACTLY_ONE);
public static final RecordField EVENT_DURATION = new SimpleRecordField(Names.EVENT_DURATION, FieldType.LONG, EXACTLY_ONE);
public static final RecordField LINEAGE_START_DATE = new SimpleRecordField(Names.LINEAGE_START_DATE, FieldType.LONG, EXACTLY_ONE);
public static final RecordField COMPONENT_ID = new SimpleRecordField(Names.COMPONENT_ID, FieldType.STRING, ZERO_OR_ONE);
public static final RecordField COMPONENT_TYPE = new SimpleRecordField(Names.COMPONENT_TYPE, FieldType.STRING, ZERO_OR_ONE);
public static final RecordField FLOWFILE_UUID = new SimpleRecordField(Names.FLOWFILE_UUID, FieldType.STRING, EXACTLY_ONE);
public static final RecordField EVENT_DETAILS = new SimpleRecordField(Names.EVENT_DETAILS, FieldType.STRING, ZERO_OR_ONE);
public static final RecordField SOURCE_QUEUE_IDENTIFIER = new SimpleRecordField(Names.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE);
// Attributes
public static final RecordField ATTRIBUTE_NAME = new SimpleRecordField(Names.ATTRIBUTE_NAME, FieldType.LONG_STRING, EXACTLY_ONE);
public static final RecordField ATTRIBUTE_VALUE_REQUIRED = new SimpleRecordField(Names.ATTRIBUTE_VALUE, FieldType.LONG_STRING, EXACTLY_ONE);
public static final RecordField ATTRIBUTE_VALUE_OPTIONAL = new SimpleRecordField(Names.ATTRIBUTE_VALUE, FieldType.LONG_STRING, ZERO_OR_ONE);
public static final RecordField PREVIOUS_ATTRIBUTES = new MapRecordField(Names.PREVIOUS_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_REQUIRED, EXACTLY_ONE);
public static final RecordField UPDATED_ATTRIBUTES = new MapRecordField(Names.UPDATED_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_OPTIONAL, EXACTLY_ONE);
// Content Claims
public static final RecordField CONTENT_CLAIM_CONTAINER = new SimpleRecordField(Names.CONTENT_CLAIM_CONTAINER, FieldType.STRING, EXACTLY_ONE);
public static final RecordField CONTENT_CLAIM_SECTION = new SimpleRecordField(Names.CONTENT_CLAIM_SECTION, FieldType.STRING, EXACTLY_ONE);
public static final RecordField CONTENT_CLAIM_IDENTIFIER = new SimpleRecordField(Names.CONTENT_CLAIM_IDENTIFIER, FieldType.STRING, EXACTLY_ONE);
public static final RecordField CONTENT_CLAIM_OFFSET = new SimpleRecordField(Names.CONTENT_CLAIM_OFFSET, FieldType.LONG, EXACTLY_ONE);
public static final RecordField CONTENT_CLAIM_SIZE = new SimpleRecordField(Names.CONTENT_CLAIM_SIZE, FieldType.LONG, EXACTLY_ONE);
public static final RecordField CURRENT_CONTENT_CLAIM = new ComplexRecordField(Names.CONTENT_CLAIM, ZERO_OR_ONE,
CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE);
public static final RecordField PREVIOUS_CONTENT_CLAIM = new ComplexRecordField(Names.PREVIOUS_CONTENT_CLAIM, ZERO_OR_ONE,
CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE);
// EventType-Specific fields
// for FORK, JOIN, CLONE, REPLAY
public static final RecordField PARENT_UUIDS = new SimpleRecordField(Names.PARENT_UUIDS, FieldType.STRING, ZERO_OR_MORE);
public static final RecordField CHILD_UUIDS = new SimpleRecordField(Names.CHILD_UUIDS, FieldType.STRING, ZERO_OR_MORE);
// for SEND/RECEIVE/FETCH
public static final RecordField TRANSIT_URI = new SimpleRecordField(Names.TRANSIT_URI, FieldType.STRING, ZERO_OR_ONE);
public static final RecordField SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = new SimpleRecordField(Names.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE);
// for ADD_INFO
public static final RecordField ALTERNATE_IDENTIFIER = new SimpleRecordField(Names.ALTERNATE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE);
public static final RecordField RELATIONSHIP = new SimpleRecordField(Names.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE);
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.schema;
import static org.apache.nifi.provenance.schema.EventRecordFields.ALTERNATE_IDENTIFIER;
import static org.apache.nifi.provenance.schema.EventRecordFields.CHILD_UUIDS;
import static org.apache.nifi.provenance.schema.EventRecordFields.COMPONENT_ID;
import static org.apache.nifi.provenance.schema.EventRecordFields.COMPONENT_TYPE;
import static org.apache.nifi.provenance.schema.EventRecordFields.CURRENT_CONTENT_CLAIM;
import static org.apache.nifi.provenance.schema.EventRecordFields.EVENT_DETAILS;
import static org.apache.nifi.provenance.schema.EventRecordFields.EVENT_DURATION;
import static org.apache.nifi.provenance.schema.EventRecordFields.EVENT_TIME;
import static org.apache.nifi.provenance.schema.EventRecordFields.EVENT_TYPE;
import static org.apache.nifi.provenance.schema.EventRecordFields.FLOWFILE_ENTRY_DATE;
import static org.apache.nifi.provenance.schema.EventRecordFields.FLOWFILE_UUID;
import static org.apache.nifi.provenance.schema.EventRecordFields.LINEAGE_START_DATE;
import static org.apache.nifi.provenance.schema.EventRecordFields.PARENT_UUIDS;
import static org.apache.nifi.provenance.schema.EventRecordFields.PREVIOUS_ATTRIBUTES;
import static org.apache.nifi.provenance.schema.EventRecordFields.PREVIOUS_CONTENT_CLAIM;
import static org.apache.nifi.provenance.schema.EventRecordFields.RECORD_IDENTIFIER;
import static org.apache.nifi.provenance.schema.EventRecordFields.RELATIONSHIP;
import static org.apache.nifi.provenance.schema.EventRecordFields.SOURCE_QUEUE_IDENTIFIER;
import static org.apache.nifi.provenance.schema.EventRecordFields.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER;
import static org.apache.nifi.provenance.schema.EventRecordFields.TRANSIT_URI;
import static org.apache.nifi.provenance.schema.EventRecordFields.UPDATED_ATTRIBUTES;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
public class ProvenanceEventSchema {
public static final RecordSchema PROVENANCE_EVENT_SCHEMA_V1 = buildSchemaV1();
private static RecordSchema buildSchemaV1() {
final List<RecordField> fields = new ArrayList<>();
fields.add(RECORD_IDENTIFIER);
fields.add(EVENT_TYPE);
fields.add(EVENT_TIME);
fields.add(FLOWFILE_ENTRY_DATE);
fields.add(EVENT_DURATION);
fields.add(LINEAGE_START_DATE);
fields.add(COMPONENT_ID);
fields.add(COMPONENT_TYPE);
fields.add(FLOWFILE_UUID);
fields.add(EVENT_DETAILS);
fields.add(PREVIOUS_ATTRIBUTES);
fields.add(UPDATED_ATTRIBUTES);
fields.add(CURRENT_CONTENT_CLAIM);
fields.add(PREVIOUS_CONTENT_CLAIM);
fields.add(SOURCE_QUEUE_IDENTIFIER);
// EventType-Specific fields
fields.add(PARENT_UUIDS); // for FORK, JOIN, CLONE, REPLAY events
fields.add(CHILD_UUIDS); // for FORK, JOIN, CLONE, REPLAY events
fields.add(TRANSIT_URI); // for SEND/RECEIVE/FETCH events
fields.add(SOURCE_SYSTEM_FLOWFILE_IDENTIFIER); // for SEND/RECEIVE events
fields.add(ALTERNATE_IDENTIFIER); // for ADD_INFO events
fields.add(RELATIONSHIP); // for ROUTE events
final RecordSchema schema = new RecordSchema(fields);
return schema;
}
}

View File

@ -0,0 +1,282 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.serialization;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.zip.GZIPInputStream;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.StandardRecordReader;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.ByteCountingInputStream;
import org.apache.nifi.stream.io.LimitingInputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class CompressableRecordReader implements RecordReader {
private static final Logger logger = LoggerFactory.getLogger(StandardRecordReader.class);
private final ByteCountingInputStream rawInputStream;
private final String filename;
private final int serializationVersion;
private final boolean compressed;
private final TocReader tocReader;
private final int headerLength;
private final int maxAttributeChars;
private DataInputStream dis;
private ByteCountingInputStream byteCountingIn;
public CompressableRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException {
this(in, filename, null, maxAttributeChars);
}
public CompressableRecordReader(final InputStream in, final String filename, final TocReader tocReader, final int maxAttributeChars) throws IOException {
logger.trace("Creating RecordReader for {}", filename);
rawInputStream = new ByteCountingInputStream(in);
this.maxAttributeChars = maxAttributeChars;
final InputStream limitedStream;
if (tocReader == null) {
limitedStream = rawInputStream;
} else {
final long offset1 = tocReader.getBlockOffset(1);
if (offset1 < 0) {
limitedStream = rawInputStream;
} else {
limitedStream = new LimitingInputStream(rawInputStream, offset1 - rawInputStream.getBytesConsumed());
}
}
final InputStream readableStream;
if (filename.endsWith(".gz")) {
readableStream = new BufferedInputStream(new GZIPInputStream(limitedStream));
compressed = true;
} else {
readableStream = new BufferedInputStream(limitedStream);
compressed = false;
}
byteCountingIn = new ByteCountingInputStream(readableStream);
dis = new DataInputStream(byteCountingIn);
final String repoClassName = dis.readUTF();
final int serializationVersion = dis.readInt();
headerLength = repoClassName.getBytes(StandardCharsets.UTF_8).length + 2 + 4; // 2 bytes for string length, 4 for integer.
this.serializationVersion = serializationVersion;
this.filename = filename;
this.tocReader = tocReader;
readHeader(dis, serializationVersion);
}
@Override
public void skipToBlock(final int blockIndex) throws IOException {
if (tocReader == null) {
throw new IllegalStateException("Cannot skip to block " + blockIndex + " for Provenance Log " + filename + " because no Table-of-Contents file was found for this Log");
}
if (blockIndex < 0) {
throw new IllegalArgumentException("Cannot skip to block " + blockIndex + " because the value is negative");
}
if (blockIndex == getBlockIndex()) {
return;
}
final long offset = tocReader.getBlockOffset(blockIndex);
if (offset < 0) {
throw new IOException("Unable to find block " + blockIndex + " in Provenance Log " + filename);
}
final long curOffset = rawInputStream.getBytesConsumed();
final long bytesToSkip = offset - curOffset;
if (bytesToSkip >= 0) {
try {
StreamUtils.skip(rawInputStream, bytesToSkip);
logger.debug("Skipped stream from offset {} to {} ({} bytes skipped)", curOffset, offset, bytesToSkip);
} catch (final IOException e) {
throw new IOException("Failed to skip to offset " + offset + " for block " + blockIndex + " of Provenance Log " + filename, e);
}
resetStreamForNextBlock();
}
}
private void resetStreamForNextBlock() throws IOException {
final InputStream limitedStream;
if (tocReader == null) {
limitedStream = rawInputStream;
} else {
final long offset = tocReader.getBlockOffset(1 + getBlockIndex());
if (offset < 0) {
limitedStream = rawInputStream;
} else {
limitedStream = new LimitingInputStream(rawInputStream, offset - rawInputStream.getBytesConsumed());
}
}
final InputStream readableStream;
if (compressed) {
readableStream = new BufferedInputStream(new GZIPInputStream(limitedStream));
} else {
readableStream = new BufferedInputStream(limitedStream);
}
byteCountingIn = new ByteCountingInputStream(readableStream, rawInputStream.getBytesConsumed());
dis = new DataInputStream(byteCountingIn);
}
@Override
public TocReader getTocReader() {
return tocReader;
}
@Override
public boolean isBlockIndexAvailable() {
return tocReader != null;
}
@Override
public int getBlockIndex() {
if (tocReader == null) {
throw new IllegalStateException("Cannot determine Block Index because no Table-of-Contents could be found for Provenance Log " + filename);
}
return tocReader.getBlockIndex(rawInputStream.getBytesConsumed());
}
@Override
public long getBytesConsumed() {
return byteCountingIn.getBytesConsumed();
}
private boolean isData() throws IOException {
byteCountingIn.mark(1);
int nextByte = byteCountingIn.read();
byteCountingIn.reset();
if (nextByte < 0) {
try {
resetStreamForNextBlock();
} catch (final EOFException eof) {
return false;
}
byteCountingIn.mark(1);
nextByte = byteCountingIn.read();
byteCountingIn.reset();
}
return nextByte >= 0;
}
@Override
public long getMaxEventId() throws IOException {
if (tocReader != null) {
final long lastBlockOffset = tocReader.getLastBlockOffset();
skipToBlock(tocReader.getBlockIndex(lastBlockOffset));
}
ProvenanceEventRecord record;
ProvenanceEventRecord lastRecord = null;
try {
while ((record = nextRecord()) != null) {
lastRecord = record;
}
} catch (final EOFException eof) {
// This can happen if we stop NIFi while the record is being written.
// This is OK, we just ignore this record. The session will not have been
// committed, so we can just process the FlowFile again.
}
return lastRecord == null ? -1L : lastRecord.getEventId();
}
@Override
public void close() throws IOException {
logger.trace("Closing Record Reader for {}", filename);
try {
dis.close();
} finally {
try {
rawInputStream.close();
} finally {
if (tocReader != null) {
tocReader.close();
}
}
}
}
@Override
public void skip(final long bytesToSkip) throws IOException {
StreamUtils.skip(dis, bytesToSkip);
}
@Override
public void skipTo(final long position) throws IOException {
// we are subtracting headerLength from the number of bytes consumed because we used to
// consider the offset of the first record "0" - now we consider it whatever position it
// it really is in the stream.
final long currentPosition = byteCountingIn.getBytesConsumed() - headerLength;
if (currentPosition == position) {
return;
}
if (currentPosition > position) {
throw new IOException("Cannot skip to byte offset " + position + " in stream because already at byte offset " + currentPosition);
}
final long toSkip = position - currentPosition;
StreamUtils.skip(dis, toSkip);
}
protected String getFilename() {
return filename;
}
protected int getMaxAttributeLength() {
return maxAttributeChars;
}
@Override
public StandardProvenanceEventRecord nextRecord() throws IOException {
if (isData()) {
return nextRecord(dis, serializationVersion);
} else {
return null;
}
}
protected abstract StandardProvenanceEventRecord nextRecord(DataInputStream in, int serializationVersion) throws IOException;
protected void readHeader(DataInputStream in, int serializationVersion) throws IOException {
}
}

View File

@ -0,0 +1,206 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.serialization;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.nifi.provenance.AbstractRecordWriter;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.ByteCountingOutputStream;
import org.apache.nifi.stream.io.DataOutputStream;
import org.apache.nifi.stream.io.GZIPOutputStream;
import org.apache.nifi.stream.io.NonCloseableOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class CompressableRecordWriter extends AbstractRecordWriter {
private static final Logger logger = LoggerFactory.getLogger(CompressableRecordWriter.class);
private final FileOutputStream fos;
private final ByteCountingOutputStream rawOutStream;
private final boolean compressed;
private final int uncompressedBlockSize;
private DataOutputStream out;
private ByteCountingOutputStream byteCountingOut;
private long lastBlockOffset = 0L;
private int recordCount = 0;
public CompressableRecordWriter(final File file, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(file, writer);
logger.trace("Creating Record Writer for {}", file.getName());
this.compressed = compressed;
this.fos = new FileOutputStream(file);
rawOutStream = new ByteCountingOutputStream(fos);
this.uncompressedBlockSize = uncompressedBlockSize;
}
public CompressableRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(null, tocWriter);
this.fos = null;
this.compressed = compressed;
this.uncompressedBlockSize = uncompressedBlockSize;
this.rawOutStream = new ByteCountingOutputStream(out);
}
@Override
public synchronized void writeHeader(final long firstEventId) throws IOException {
if (isDirty()) {
throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository");
}
try {
lastBlockOffset = rawOutStream.getBytesWritten();
resetWriteStream(firstEventId);
out.writeUTF(getSerializationName());
out.writeInt(getSerializationVersion());
writeHeader(firstEventId, out);
out.flush();
lastBlockOffset = rawOutStream.getBytesWritten();
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
/**
* Resets the streams to prepare for a new block
*
* @param eventId the first id that will be written to the new block
* @throws IOException if unable to flush/close the current streams properly
*/
private void resetWriteStream(final long eventId) throws IOException {
try {
if (out != null) {
out.flush();
}
final long byteOffset = (byteCountingOut == null) ? rawOutStream.getBytesWritten() : byteCountingOut.getBytesWritten();
final TocWriter tocWriter = getTocWriter();
final OutputStream writableStream;
if (compressed) {
// because of the way that GZIPOutputStream works, we need to call close() on it in order for it
// to write its trailing bytes. But we don't want to close the underlying OutputStream, so we wrap
// the underlying OutputStream in a NonCloseableOutputStream
// We don't have to check if the writer is dirty because we will have already checked before calling this method.
if (out != null) {
out.close();
}
if (tocWriter != null) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId);
}
writableStream = new BufferedOutputStream(new GZIPOutputStream(new NonCloseableOutputStream(rawOutStream), 1), 65536);
} else {
if (tocWriter != null) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId);
}
writableStream = new BufferedOutputStream(rawOutStream, 65536);
}
this.byteCountingOut = new ByteCountingOutputStream(writableStream, byteOffset);
this.out = new DataOutputStream(byteCountingOut);
resetDirtyFlag();
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
@Override
public long writeRecord(final ProvenanceEventRecord record, final long recordIdentifier) throws IOException {
if (isDirty()) {
throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository");
}
try {
final long startBytes = byteCountingOut.getBytesWritten();
// add a new block to the TOC if needed.
if (getTocWriter() != null && (startBytes - lastBlockOffset >= uncompressedBlockSize)) {
lastBlockOffset = startBytes;
if (compressed) {
// because of the way that GZIPOutputStream works, we need to call close() on it in order for it
// to write its trailing bytes. But we don't want to close the underlying OutputStream, so we wrap
// the underlying OutputStream in a NonCloseableOutputStream
resetWriteStream(recordIdentifier);
}
}
writeRecord(record, recordIdentifier, out);
recordCount++;
return byteCountingOut.getBytesWritten() - startBytes;
} catch (final IOException ioe) {
markDirty();
throw ioe;
}
}
@Override
public void flush() throws IOException {
out.flush();
}
@Override
public synchronized int getRecordsWritten() {
return recordCount;
}
@Override
protected OutputStream getBufferedOutputStream() {
return out;
}
@Override
protected OutputStream getUnderlyingOutputStream() {
return fos;
}
@Override
protected void syncUnderlyingOutputStream() throws IOException {
if (fos != null) {
fos.getFD().sync();
}
}
protected abstract void writeRecord(final ProvenanceEventRecord event, final long eventId, final DataOutputStream out) throws IOException;
protected abstract void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException;
protected abstract int getSerializationVersion();
protected abstract String getSerializationName();
}

View File

@ -14,50 +14,59 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository.io;
package org.apache.nifi.provenance.serialization;
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicLong;
public class ByteCountingOutputStream extends OutputStream {
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.toc.TocReader;
private final AtomicLong bytesWrittenHolder;
private final OutputStream out;
public ByteCountingOutputStream(final OutputStream out, final AtomicLong longHolder) {
this.out = out;
this.bytesWrittenHolder = longHolder;
}
@Override
public void write(int b) throws IOException {
out.write(b);
bytesWrittenHolder.getAndIncrement();
}
@Override
public void write(byte[] b) throws IOException {
write(b, 0, b.length);
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
out.write(b, off, len);
bytesWrittenHolder.getAndAdd(len);
}
public long getBytesWritten() {
return bytesWrittenHolder.get();
}
@Override
public void flush() throws IOException {
out.flush();
}
public class EmptyRecordReader implements RecordReader {
@Override
public void close() throws IOException {
out.close();
}
@Override
public StandardProvenanceEventRecord nextRecord() throws IOException {
return null;
}
@Override
public void skip(long bytesToSkip) throws IOException {
}
@Override
public void skipTo(long position) throws IOException {
}
@Override
public void skipToBlock(int blockIndex) throws IOException {
}
@Override
public int getBlockIndex() {
return 0;
}
@Override
public boolean isBlockIndexAvailable() {
return false;
}
@Override
public TocReader getTocReader() {
return null;
}
@Override
public long getBytesConsumed() {
return 0;
}
@Override
public long getMaxEventId() throws IOException {
return 0;
}
}

View File

@ -16,6 +16,9 @@
*/
package org.apache.nifi.provenance.serialization;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
@ -23,8 +26,12 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Path;
import java.util.Collection;
import java.util.zip.GZIPInputStream;
import org.apache.nifi.provenance.ByteArraySchemaRecordReader;
import org.apache.nifi.provenance.ByteArraySchemaRecordWriter;
import org.apache.nifi.provenance.StandardRecordReader;
import org.apache.nifi.provenance.StandardRecordWriter;
import org.apache.nifi.provenance.lucene.LuceneUtil;
import org.apache.nifi.provenance.toc.StandardTocReader;
import org.apache.nifi.provenance.toc.TocReader;
@ -101,11 +108,39 @@ public class RecordReaders {
}
final File tocFile = TocUtil.getTocFile(file);
if ( tocFile.exists() ) {
final TocReader tocReader = new StandardTocReader(tocFile);
return new StandardRecordReader(fis, filename, tocReader, maxAttributeChars);
} else {
return new StandardRecordReader(fis, filename, maxAttributeChars);
final InputStream bufferedInStream = new BufferedInputStream(fis);
final String serializationName;
try {
bufferedInStream.mark(4096);
final InputStream in = filename.endsWith(".gz") ? new GZIPInputStream(bufferedInStream) : bufferedInStream;
final DataInputStream dis = new DataInputStream(in);
serializationName = dis.readUTF();
bufferedInStream.reset();
} catch (final EOFException eof) {
return new EmptyRecordReader();
}
switch (serializationName) {
case StandardRecordWriter.SERIALIZATION_NAME: {
if (tocFile.exists()) {
final TocReader tocReader = new StandardTocReader(tocFile);
return new StandardRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars);
} else {
return new StandardRecordReader(bufferedInStream, filename, maxAttributeChars);
}
}
case ByteArraySchemaRecordWriter.SERIALIZATION_NAME: {
if (tocFile.exists()) {
final TocReader tocReader = new StandardTocReader(tocFile);
return new ByteArraySchemaRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars);
} else {
return new ByteArraySchemaRecordReader(bufferedInStream, filename, maxAttributeChars);
}
}
default: {
throw new IOException("Unable to read data from file " + file + " because the file was written using an unknown Serializer: " + serializationName);
}
}
} catch (final IOException ioe) {
if ( fis != null ) {

View File

@ -43,6 +43,13 @@ public interface RecordWriter extends Closeable {
*/
long writeRecord(ProvenanceEventRecord record, long recordIdentifier) throws IOException;
/**
* Flushes any data that is held in a buffer to the underlying storage mechanism
*
* @throws IOException if unable to flush the bytes
*/
void flush() throws IOException;
/**
* @return the number of Records that have been written to this RecordWriter
*/

View File

@ -19,7 +19,7 @@ package org.apache.nifi.provenance.serialization;
import java.io.File;
import java.io.IOException;
import org.apache.nifi.provenance.StandardRecordWriter;
import org.apache.nifi.provenance.ByteArraySchemaRecordWriter;
import org.apache.nifi.provenance.toc.StandardTocWriter;
import org.apache.nifi.provenance.toc.TocUtil;
import org.apache.nifi.provenance.toc.TocWriter;
@ -27,13 +27,13 @@ import org.apache.nifi.provenance.toc.TocWriter;
public class RecordWriters {
private static final int DEFAULT_COMPRESSION_BLOCK_SIZE = 1024 * 1024; // 1 MB
public static RecordWriter newRecordWriter(final File file, final boolean compressed, final boolean createToc) throws IOException {
return newRecordWriter(file, compressed, createToc, DEFAULT_COMPRESSION_BLOCK_SIZE);
public static RecordWriter newSchemaRecordWriter(final File file, final boolean compressed, final boolean createToc) throws IOException {
return newSchemaRecordWriter(file, compressed, createToc, DEFAULT_COMPRESSION_BLOCK_SIZE);
}
public static RecordWriter newRecordWriter(final File file, final boolean compressed, final boolean createToc, final int compressionBlockBytes) throws IOException {
public static RecordWriter newSchemaRecordWriter(final File file, final boolean compressed, final boolean createToc, final int compressionBlockBytes) throws IOException {
final TocWriter tocWriter = createToc ? new StandardTocWriter(TocUtil.getTocFile(file), false, false) : null;
return new StandardRecordWriter(file, tocWriter, compressed, compressionBlockBytes);
return new ByteArraySchemaRecordWriter(file, tocWriter, compressed, compressionBlockBytes);
}
}

View File

@ -0,0 +1,204 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import static org.apache.nifi.provenance.TestUtil.createFlowFile;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import org.apache.nifi.provenance.serialization.RecordReader;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.StandardTocReader;
import org.apache.nifi.provenance.toc.StandardTocWriter;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.provenance.toc.TocUtil;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.util.file.FileUtils;
import org.junit.BeforeClass;
import org.junit.Test;
public abstract class AbstractTestRecordReaderWriter {
@BeforeClass
public static void setLogLevel() {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", "INFO");
}
protected ProvenanceEventRecord createEvent() {
final Map<String, String> attributes = new HashMap<>();
attributes.put("filename", "1.txt");
attributes.put("uuid", UUID.randomUUID().toString());
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
builder.setEventTime(System.currentTimeMillis());
builder.setEventType(ProvenanceEventType.RECEIVE);
builder.setTransitUri("nifi://unit-test");
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
builder.setComponentId("1234");
builder.setComponentType("dummy processor");
final ProvenanceEventRecord record = builder.build();
return record;
}
@Test
public void testSimpleWriteWithToc() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
final RecordWriter writer = createWriter(journalFile, tocWriter, false, 1024 * 1024);
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 1L);
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) {
assertEquals(0, reader.getBlockIndex());
reader.skipToBlock(0);
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
assertEquals("nifi://unit-test", recovered.getTransitUri());
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
@Test
public void testSingleRecordCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192);
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 1L);
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) {
assertEquals(0, reader.getBlockIndex());
reader.skipToBlock(0);
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
assertEquals("nifi://unit-test", recovered.getTransitUri());
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
@Test
public void testMultipleRecordsSameBlockCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
// new record each 1 MB of uncompressed data
final RecordWriter writer = createWriter(journalFile, tocWriter, true, 1024 * 1024);
writer.writeHeader(1L);
for (int i = 0; i < 10; i++) {
writer.writeRecord(createEvent(), i);
}
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) {
for (int i = 0; i < 10; i++) {
assertEquals(0, reader.getBlockIndex());
// call skipToBlock half the time to ensure that we can; avoid calling it
// the other half of the time to ensure that it's okay.
if (i <= 5) {
reader.skipToBlock(0);
}
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
assertEquals("nifi://unit-test", recovered.getTransitUri());
}
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
@Test
public void testMultipleRecordsMultipleBlocksCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
// new block each 10 bytes
final RecordWriter writer = createWriter(journalFile, tocWriter, true, 100);
writer.writeHeader(1L);
for (int i = 0; i < 10; i++) {
writer.writeRecord(createEvent(), i);
}
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) {
for (int i = 0; i < 10; i++) {
final StandardProvenanceEventRecord recovered = reader.nextRecord();
System.out.println(recovered);
assertNotNull(recovered);
assertEquals(i, recovered.getEventId());
assertEquals("nifi://unit-test", recovered.getTransitUri());
final Map<String, String> updatedAttrs = recovered.getUpdatedAttributes();
assertNotNull(updatedAttrs);
assertEquals(2, updatedAttrs.size());
assertEquals("1.txt", updatedAttrs.get("filename"));
assertTrue(updatedAttrs.containsKey("uuid"));
}
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
protected abstract RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException;
protected abstract RecordReader createReader(InputStream in, String journalFilename, TocReader tocReader, int maxAttributeSize) throws IOException;
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import java.io.IOException;
import java.io.InputStream;
public class LoopingInputStream extends InputStream {
private final byte[] buffer;
private int index;
private final byte[] header;
private int headerIndex;
private boolean headerComplete = false;
public LoopingInputStream(final byte[] header, final byte[] toRepeat) {
this.header = header;
this.buffer = toRepeat;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
@Override
public int read() throws IOException {
if (headerComplete) {
final byte nextByte = buffer[index++];
if (index >= buffer.length) {
index = 0;
}
final int returnValue = nextByte & 0xFF;
return returnValue;
} else {
final byte nextByte = header[headerIndex++];
if (headerIndex >= header.length) {
headerComplete = true;
}
final int returnValue = nextByte & 0xFF;
return returnValue;
}
}
@Override
public int read(final byte[] b, final int off, final int len) throws IOException {
if (headerComplete) {
final int toRead = Math.min(len, buffer.length - index);
System.arraycopy(buffer, index, b, off, toRead);
index += toRead;
if (index >= buffer.length) {
index = 0;
}
return toRead;
} else {
final int toRead = Math.min(len, header.length - headerIndex);
System.arraycopy(header, headerIndex, b, off, toRead);
headerIndex += toRead;
if (headerIndex >= header.length) {
headerComplete = true;
}
return toRead;
}
}
@Override
public int available() throws IOException {
return 1;
}
@Override
public void close() throws IOException {
}
}

View File

@ -38,7 +38,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
@ -86,9 +85,6 @@ import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -1161,6 +1157,7 @@ public class TestPersistentProvenanceRepository {
}
@Test
@Ignore("This test relies too much on timing of background events by using Thread.sleep().")
public void testIndexDirectoryRemoved() throws InterruptedException, IOException, ParseException {
final RepositoryConfiguration config = createConfiguration();
config.setMaxRecordLife(5, TimeUnit.MINUTES);
@ -1198,6 +1195,10 @@ public class TestPersistentProvenanceRepository {
Thread.sleep(2000L);
final FileFilter indexFileFilter = file -> file.getName().startsWith("index");
final int numIndexDirs = config.getStorageDirectories().get(0).listFiles(indexFileFilter).length;
assertEquals(1, numIndexDirs);
// add more records so that we will create a new index
final long secondBatchStartTime = System.currentTimeMillis();
for (int i = 0; i < 10; i++) {
@ -1221,12 +1222,6 @@ public class TestPersistentProvenanceRepository {
assertEquals(20, result.getMatchingEvents().size());
// Ensure index directories exists
final FileFilter indexFileFilter = new FileFilter() {
@Override
public boolean accept(File pathname) {
return pathname.getName().startsWith("index");
}
};
File[] indexDirs = config.getStorageDirectories().get(0).listFiles(indexFileFilter);
assertEquals(2, indexDirs.length);
@ -1777,8 +1772,12 @@ public class TestPersistentProvenanceRepository {
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
repo.initialize(getEventReporter(), null, null);
final String maxLengthChars = "12345678901234567890123456789012345678901234567890";
final Map<String, String> attributes = new HashMap<>();
attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345");
attributes.put("51chars", "123456789012345678901234567890123456789012345678901");
attributes.put("50chars", "12345678901234567890123456789012345678901234567890");
attributes.put("49chars", "1234567890123456789012345678901234567890123456789");
attributes.put("nullChar", null);
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
@ -1797,11 +1796,14 @@ public class TestPersistentProvenanceRepository {
final ProvenanceEventRecord retrieved = repo.getEvent(0L, null);
assertNotNull(retrieved);
assertEquals("12345678-0000-0000-0000-012345678912", retrieved.getAttributes().get("uuid"));
assertEquals("12345678901234567890123456789012345678901234567890", retrieved.getAttributes().get("75chars"));
assertEquals(maxLengthChars, retrieved.getAttributes().get("75chars"));
assertEquals(maxLengthChars, retrieved.getAttributes().get("51chars"));
assertEquals(maxLengthChars, retrieved.getAttributes().get("50chars"));
assertEquals(maxLengthChars.substring(0, 49), retrieved.getAttributes().get("49chars"));
}
@Test(timeout=5000)
@Test(timeout = 15000)
public void testExceptionOnIndex() throws IOException {
final RepositoryConfiguration config = createConfiguration();
config.setMaxAttributeChars(50);
@ -1914,112 +1916,6 @@ public class TestPersistentProvenanceRepository {
}
@Test
public void testBehaviorOnOutOfMemory() throws IOException, InterruptedException {
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(3, TimeUnit.MINUTES);
config.setJournalCount(4);
// Create a repository that overrides the createWriters() method so that we can return writers that will throw
// OutOfMemoryError where we want to
final AtomicBoolean causeOOME = new AtomicBoolean(false);
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
@Override
protected RecordWriter[] createWriters(RepositoryConfiguration config, long initialRecordId) throws IOException {
final RecordWriter[] recordWriters = super.createWriters(config, initialRecordId);
// Spy on each of the writers so that a call to writeUUID throws an OutOfMemoryError if we set the
// causeOOME flag to true
final StandardRecordWriter[] spiedWriters = new StandardRecordWriter[recordWriters.length];
for (int i = 0; i < recordWriters.length; i++) {
final StandardRecordWriter writer = (StandardRecordWriter) recordWriters[i];
spiedWriters[i] = Mockito.spy(writer);
Mockito.doAnswer(new Answer<Object>() {
@Override
public Object answer(final InvocationOnMock invocation) throws Throwable {
if (causeOOME.get()) {
throw new OutOfMemoryError();
} else {
writer.writeUUID(invocation.getArgumentAt(0, DataOutputStream.class), invocation.getArgumentAt(1, String.class));
}
return null;
}
}).when(spiedWriters[i]).writeUUID(Mockito.any(DataOutputStream.class), Mockito.any(String.class));
}
// return the writers that we are spying on
return spiedWriters;
}
};
repo.initialize(getEventReporter(), null, null);
final Map<String, String> attributes = new HashMap<>();
attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345");
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
builder.setEventTime(System.currentTimeMillis());
builder.setEventType(ProvenanceEventType.RECEIVE);
builder.setTransitUri("nifi://unit-test");
attributes.put("uuid", "12345678-0000-0000-0000-012345678912");
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
builder.setComponentId("1234");
builder.setComponentType("dummy processor");
// first make sure that we are able to write to the repo successfully.
for (int i = 0; i < 4; i++) {
final ProvenanceEventRecord record = builder.build();
repo.registerEvent(record);
}
// cause OOME to occur
causeOOME.set(true);
// write 4 times to make sure that we mark all partitions as dirty
for (int i = 0; i < 4; i++) {
final ProvenanceEventRecord record = builder.build();
try {
repo.registerEvent(record);
Assert.fail("Expected OutOfMemoryError but was able to register event");
} catch (final OutOfMemoryError oome) {
}
}
// now that all partitions are dirty, ensure that as we keep trying to write, we get an IllegalStateException
// and that we don't corrupt the repository by writing partial records
for (int i = 0; i < 8; i++) {
final ProvenanceEventRecord record = builder.build();
try {
repo.registerEvent(record);
Assert.fail("Expected OutOfMemoryError but was able to register event");
} catch (final IllegalStateException ise) {
}
}
// close repo so that we can create a new one to recover records
repo.close();
// make sure we can recover
final PersistentProvenanceRepository recoveryRepo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
@Override
protected Set<File> recoverJournalFiles() throws IOException {
try {
return super.recoverJournalFiles();
} catch (final IOException ioe) {
Assert.fail("Failed to recover properly");
return null;
}
}
};
try {
recoveryRepo.initialize(getEventReporter(), null, null);
} finally {
recoveryRepo.close();
}
}
private static class ReportedEvent {
private final Severity severity;
private final String category;

View File

@ -0,0 +1,315 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import org.apache.nifi.provenance.schema.EventRecord;
import org.apache.nifi.provenance.schema.EventRecordFields;
import org.apache.nifi.provenance.schema.ProvenanceEventSchema;
import org.apache.nifi.provenance.serialization.RecordReader;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.NopTocWriter;
import org.apache.nifi.provenance.toc.StandardTocReader;
import org.apache.nifi.provenance.toc.StandardTocWriter;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.provenance.toc.TocUtil;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.repository.schema.FieldMapRecord;
import org.apache.nifi.repository.schema.FieldType;
import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.Repetition;
import org.apache.nifi.repository.schema.SimpleRecordField;
import org.apache.nifi.stream.io.DataOutputStream;
import org.apache.nifi.stream.io.NullOutputStream;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter {
private File journalFile;
private File tocFile;
@Before
public void setup() {
journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testFieldAddedToSchema");
tocFile = TocUtil.getTocFile(journalFile);
}
@Test
public void testFieldAddedToSchema() throws IOException {
final RecordField unitTestField = new SimpleRecordField("Unit Test Field", FieldType.STRING, Repetition.EXACTLY_ONE);
final Consumer<List<RecordField>> schemaModifier = fields -> fields.add(unitTestField);
final Map<RecordField, Object> toAdd = new HashMap<>();
toAdd.put(unitTestField, "hello");
try (final ByteArraySchemaRecordWriter writer = createSchemaWriter(schemaModifier, toAdd)) {
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 3L);
writer.writeRecord(createEvent(), 3L);
}
try (final InputStream in = new FileInputStream(journalFile);
final TocReader tocReader = new StandardTocReader(tocFile);
final RecordReader reader = createReader(in, journalFile.getName(), tocReader, 10000)) {
for (int i = 0; i < 2; i++) {
final StandardProvenanceEventRecord event = reader.nextRecord();
assertNotNull(event);
assertEquals(3L, event.getEventId());
assertEquals("1234", event.getComponentId());
assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
assertNotNull(event.getUpdatedAttributes());
assertFalse(event.getUpdatedAttributes().isEmpty());
}
}
}
@Test
public void testFieldRemovedFromSchema() throws IOException {
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
try {
// Create a schema that has the fields modified
final RecordSchema schemaV1 = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1;
final List<RecordField> fields = new ArrayList<>(schemaV1.getFields());
fields.remove(new SimpleRecordField(EventRecordFields.Names.UPDATED_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE));
fields.remove(new SimpleRecordField(EventRecordFields.Names.PREVIOUS_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE));
final RecordSchema recordSchema = new RecordSchema(fields);
// Create a record writer whose schema does not contain updated attributes or previous attributes.
// This means that we must also override the method that writes out attributes so that we are able
// to avoid actually writing them out.
final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, tocWriter, false, 0) {
@Override
public void writeHeader(long firstEventId, DataOutputStream out) throws IOException {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
recordSchema.writeTo(baos);
out.writeInt(baos.size());
baos.writeTo(out);
}
@Override
protected Record createRecord(final ProvenanceEventRecord event, final long eventId) {
final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields());
return new EventRecord(event, eventId, recordSchema, contentClaimSchema);
}
};
try {
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 3L);
writer.writeRecord(createEvent(), 3L);
} finally {
writer.close();
}
} finally {
tocWriter.close();
}
// Read the records in and make sure that they have the info that we expect.
try (final InputStream in = new FileInputStream(journalFile);
final TocReader tocReader = new StandardTocReader(tocFile);
final RecordReader reader = createReader(in, journalFile.getName(), tocReader, 10000)) {
for (int i = 0; i < 2; i++) {
final StandardProvenanceEventRecord event = reader.nextRecord();
assertNotNull(event);
assertEquals(3L, event.getEventId());
assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
// We will still have a Map<String, String> for updated attributes because the
// Provenance Event Builder will create an empty map.
assertNotNull(event.getUpdatedAttributes());
assertTrue(event.getUpdatedAttributes().isEmpty());
}
}
}
/**
* Creates a SchemaRecordWriter that uses a modified schema
*
* @param fieldModifier the callback for modifying the schema
* @return a SchemaRecordWriter that uses the modified schema
* @throws IOException if unable to create the writer
*/
private ByteArraySchemaRecordWriter createSchemaWriter(final Consumer<List<RecordField>> fieldModifier, final Map<RecordField, Object> fieldsToAdd) throws IOException {
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
// Create a schema that has the fields modified
final RecordSchema schemaV1 = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1;
final List<RecordField> fields = new ArrayList<>(schemaV1.getFields());
fieldModifier.accept(fields);
final RecordSchema recordSchema = new RecordSchema(fields);
final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields());
final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, tocWriter, false, 0) {
@Override
public void writeHeader(long firstEventId, DataOutputStream out) throws IOException {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
recordSchema.writeTo(baos);
out.writeInt(baos.size());
baos.writeTo(out);
}
@Override
protected Record createRecord(final ProvenanceEventRecord event, final long eventId) {
final Map<RecordField, Object> values = new HashMap<>();
final EventRecord eventRecord = new EventRecord(event, eventId, recordSchema, contentClaimSchema);
for (final RecordField field : recordSchema.getFields()) {
final Object value = eventRecord.getFieldValue(field);
values.put(field, value);
}
values.putAll(fieldsToAdd);
return new FieldMapRecord(values, recordSchema);
}
};
return writer;
}
@Test
@Ignore("For local testing only")
public void testWritePerformance() throws IOException {
// This is a simple micro-benchmarking test so that we can determine how fast the serialization/deserialization is before
// making significant changes. This allows us to ensure that changes that we make do not have significant adverse effects
// on performance of the repository.
final ProvenanceEventRecord event = createEvent();
final TocWriter tocWriter = new NopTocWriter();
final int numEvents = 10_000_000;
final long startNanos = System.nanoTime();
try (final OutputStream nullOut = new NullOutputStream();
final RecordWriter writer = new ByteArraySchemaRecordWriter(nullOut, tocWriter, false, 0)) {
writer.writeHeader(0L);
for (int i = 0; i < numEvents; i++) {
writer.writeRecord(event, i);
}
}
final long nanos = System.nanoTime() - startNanos;
final long millis = TimeUnit.NANOSECONDS.toMillis(nanos);
System.out.println("Took " + millis + " millis to write " + numEvents + " events");
}
@Test
@Ignore("For local performance testing only")
public void testReadPerformance() throws IOException, InterruptedException {
// This is a simple micro-benchmarking test so that we can determine how fast the serialization/deserialization is before
// making significant changes. This allows us to ensure that changes that we make do not have significant adverse effects
// on performance of the repository.
final ProvenanceEventRecord event = createEvent();
final TocReader tocReader = null;
final byte[] header;
try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream();
final DataOutputStream out = new DataOutputStream(headerOut)) {
final RecordWriter schemaWriter = new ByteArraySchemaRecordWriter(out, null, false, 0);
schemaWriter.writeHeader(1L);
header = headerOut.toByteArray();
}
final byte[] serializedRecord;
try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream();
final RecordWriter writer = new ByteArraySchemaRecordWriter(headerOut, null, false, 0)) {
writer.writeHeader(1L);
headerOut.reset();
writer.writeRecord(event, 1L);
writer.flush();
serializedRecord = headerOut.toByteArray();
}
final int numEvents = 10_000_000;
final int recordBytes = serializedRecord.length;
final long totalRecordBytes = (long) recordBytes * (long) numEvents;
final long startNanos = System.nanoTime();
try (final InputStream in = new LoopingInputStream(header, serializedRecord);
final RecordReader reader = new ByteArraySchemaRecordReader(in, "filename", tocReader, 100000)) {
for (int i = 0; i < numEvents; i++) {
reader.nextRecord();
}
}
final long nanos = System.nanoTime() - startNanos;
final long millis = TimeUnit.NANOSECONDS.toMillis(nanos);
final double seconds = millis / 1000D;
final long bytesPerSecond = (long) (totalRecordBytes / seconds);
final long megaBytesPerSecond = bytesPerSecond / 1024 / 1024;
System.out.println("Took " + millis + " millis to read " + numEvents + " events or " + megaBytesPerSecond + " MB/sec");
}
@Override
protected RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException {
return new ByteArraySchemaRecordWriter(file, tocWriter, compressed, uncompressedBlockSize);
}
@Override
protected RecordReader createReader(InputStream in, String journalFilename, TocReader tocReader, int maxAttributeSize) throws IOException {
final ByteArraySchemaRecordReader reader = new ByteArraySchemaRecordReader(in, journalFilename, tocReader, maxAttributeSize);
return reader;
}
private static interface WriteRecordInterceptor {
void writeRawRecord(ProvenanceEventRecord event, long recordIdentifier, DataOutputStream out) throws IOException;
}
private static WriteRecordInterceptor NOP_INTERCEPTOR = (event, id, out) -> {};
private static WriteRecordInterceptor WRITE_DUMMY_STRING_INTERCEPTOR = (event, id, out) -> out.writeUTF("hello");
}

View File

@ -16,174 +16,106 @@
*/
package org.apache.nifi.provenance;
import static org.apache.nifi.provenance.TestUtil.createFlowFile;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.provenance.toc.StandardTocReader;
import org.apache.nifi.provenance.toc.StandardTocWriter;
import org.apache.nifi.provenance.serialization.RecordReader;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.NopTocWriter;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.provenance.toc.TocUtil;
import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.util.file.FileUtils;
import org.junit.BeforeClass;
import org.apache.nifi.stream.io.DataOutputStream;
import org.apache.nifi.stream.io.NullOutputStream;
import org.junit.Ignore;
import org.junit.Test;
public class TestStandardRecordReaderWriter {
@BeforeClass
public static void setLogLevel() {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", "DEBUG");
}
private ProvenanceEventRecord createEvent() {
final Map<String, String> attributes = new HashMap<>();
attributes.put("filename", "1.txt");
attributes.put("uuid", UUID.randomUUID().toString());
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
builder.setEventTime(System.currentTimeMillis());
builder.setEventType(ProvenanceEventType.RECEIVE);
builder.setTransitUri("nifi://unit-test");
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
builder.setComponentId("1234");
builder.setComponentType("dummy processor");
final ProvenanceEventRecord record = builder.build();
return record;
}
@Test
public void testSimpleWriteWithToc() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
final StandardRecordWriter writer = new StandardRecordWriter(journalFile, tocWriter, false, 1024 * 1024);
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 1L);
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) {
assertEquals(0, reader.getBlockIndex());
reader.skipToBlock(0);
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
assertEquals("nifi://unit-test", recovered.getTransitUri());
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
public class TestStandardRecordReaderWriter extends AbstractTestRecordReaderWriter {
@Test
public void testSingleRecordCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
final StandardRecordWriter writer = new StandardRecordWriter(journalFile, tocWriter, true, 100);
@Ignore("For local testing only")
public void testWritePerformance() throws IOException {
// This is a simple micro-benchmarking test so that we can determine how fast the serialization/deserialization is before
// making significant changes. This allows us to ensure that changes that we make do not have significant adverse effects
// on performance of the repository.
final ProvenanceEventRecord event = createEvent();
writer.writeHeader(1L);
writer.writeRecord(createEvent(), 1L);
writer.close();
final TocWriter tocWriter = new NopTocWriter();
final TocReader tocReader = new StandardTocReader(tocFile);
final int numEvents = 10_000_000;
final long startNanos = System.nanoTime();
try (final OutputStream nullOut = new NullOutputStream();
final RecordWriter writer = new StandardRecordWriter(nullOut, tocWriter, false, 100000)) {
try (final FileInputStream fis = new FileInputStream(journalFile);
final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) {
assertEquals(0, reader.getBlockIndex());
reader.skipToBlock(0);
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
writer.writeHeader(0L);
assertEquals("nifi://unit-test", recovered.getTransitUri());
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
}
@Test
public void testMultipleRecordsSameBlockCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
// new record each 1 MB of uncompressed data
final StandardRecordWriter writer = new StandardRecordWriter(journalFile, tocWriter, true, 1024 * 1024);
writer.writeHeader(1L);
for (int i=0; i < 10; i++) {
writer.writeRecord(createEvent(), i);
}
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
try (final FileInputStream fis = new FileInputStream(journalFile);
final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) {
for (int i=0; i < 10; i++) {
assertEquals(0, reader.getBlockIndex());
// call skipToBlock half the time to ensure that we can; avoid calling it
// the other half of the time to ensure that it's okay.
if (i <= 5) {
reader.skipToBlock(0);
}
final StandardProvenanceEventRecord recovered = reader.nextRecord();
assertNotNull(recovered);
assertEquals("nifi://unit-test", recovered.getTransitUri());
for (int i = 0; i < numEvents; i++) {
writer.writeRecord(event, i);
}
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
final long nanos = System.nanoTime() - startNanos;
final long millis = TimeUnit.NANOSECONDS.toMillis(nanos);
System.out.println("Took " + millis + " millis to write " + numEvents + " events");
}
@Test
public void testMultipleRecordsMultipleBlocksCompressed() throws IOException {
final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz");
final File tocFile = TocUtil.getTocFile(journalFile);
final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false);
// new block each 10 bytes
final StandardRecordWriter writer = new StandardRecordWriter(journalFile, tocWriter, true, 100);
@Ignore("For local testing only")
public void testReadPerformance() throws IOException {
// This is a simple micro-benchmarking test so that we can determine how fast the serialization/deserialization is before
// making significant changes. This allows us to ensure that changes that we make do not have significant adverse effects
// on performance of the repository.
final ProvenanceEventRecord event = createEvent();
writer.writeHeader(1L);
for (int i=0; i < 10; i++) {
writer.writeRecord(createEvent(), i);
final TocReader tocReader = null;
final byte[] header;
try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream();
final DataOutputStream out = new DataOutputStream(headerOut)) {
out.writeUTF(PersistentProvenanceRepository.class.getName());
out.writeInt(9);
header = headerOut.toByteArray();
}
writer.close();
final TocReader tocReader = new StandardTocReader(tocFile);
final byte[] serializedRecord;
try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream();
final StandardRecordWriter writer = new StandardRecordWriter(headerOut, null, false, 0)) {
try (final FileInputStream fis = new FileInputStream(journalFile);
final StandardRecordReader reader = new StandardRecordReader(fis, journalFile.getName(), tocReader, 2048)) {
for (int i=0; i < 10; i++) {
final StandardProvenanceEventRecord recovered = reader.nextRecord();
System.out.println(recovered);
assertNotNull(recovered);
assertEquals(i, recovered.getEventId());
assertEquals("nifi://unit-test", recovered.getTransitUri());
writer.writeHeader(1L);
headerOut.reset();
writer.writeRecord(event, 1L);
writer.flush();
serializedRecord = headerOut.toByteArray();
}
final int numEvents = 10_000_000;
final long startNanos = System.nanoTime();
try (final InputStream in = new LoopingInputStream(header, serializedRecord);
final RecordReader reader = new StandardRecordReader(in, "filename", tocReader, 100000)) {
for (int i = 0; i < numEvents; i++) {
reader.nextRecord();
}
assertNull(reader.nextRecord());
}
FileUtils.deleteFile(journalFile.getParentFile(), true);
final long nanos = System.nanoTime() - startNanos;
final long millis = TimeUnit.NANOSECONDS.toMillis(nanos);
System.out.println("Took " + millis + " millis to read " + numEvents + " events");
}
@Override
protected RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException {
return new StandardRecordWriter(file, tocWriter, compressed, uncompressedBlockSize);
}
@Override
protected RecordReader createReader(InputStream in, String journalFilename, TocReader tocReader, int maxAttributeSize) throws IOException {
return new StandardRecordReader(in, journalFilename, tocReader, maxAttributeSize);
}
}

View File

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.provenance.toc;
import java.io.File;
import java.io.IOException;
public class NopTocWriter implements TocWriter {
private int blockIndex;
@Override
public void close() throws IOException {
}
@Override
public void addBlockOffset(long offset, long firstEventId) throws IOException {
blockIndex++;
}
@Override
public int getCurrentBlockIndex() {
return blockIndex;
}
@Override
public File getFile() {
return null;
}
@Override
public void sync() throws IOException {
}
}

View File

@ -1305,6 +1305,11 @@ language governing permissions and limitations under the License. -->
<artifactId>nifi-processor-utils</artifactId>
<version>1.1.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-schema-utils</artifactId>
<version>1.1.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-hadoop-utils</artifactId>