NIFI-5244 Fixed a bug in MockSchemaRegistry that prevented it from loading using the schema name strategy.

This closes #2742.

Signed-off-by: Bryan Bende <bbende@apache.org>
This commit is contained in:
Mike Thomsen 2018-05-27 16:05:03 -04:00 committed by Bryan Bende
parent 65beec39bb
commit 688f906a73
No known key found for this signature in database
GPG Key ID: A0DDA9ED50711C39
3 changed files with 121 additions and 1 deletions

View File

@ -46,5 +46,79 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-avro-record-utils</artifactId>
<version>1.7.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>1.5</version>
<executions>
<execution>
<id>add-source</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/groovy</source>
</sources>
</configuration>
</execution>
<execution>
<id>add-test-source</id>
<phase>generate-test-sources</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>src/test/groovy</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<executions>
<!-- Only run for tests -->
<execution>
<id>groovy-tests</id>
<goals>
<goal>testCompile</goal>
</goals>
<configuration>
<compilerId>groovy-eclipse-compiler</compilerId>
</configuration>
</execution>
</executions>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
<dependencies>
<dependency>
<groupId>org.codehaus.groovy</groupId>
<artifactId>groovy-eclipse-compiler</artifactId>
<version>2.9.2-01</version>
</dependency>
<dependency>
<groupId>org.codehaus.groovy</groupId>
<artifactId>groovy-eclipse-batch</artifactId>
<version>2.4.3-01</version>
</dependency>
</dependencies>
</plugin>
</plugins>
</build>
</project>

View File

@ -46,7 +46,7 @@ public class MockSchemaRegistry extends AbstractControllerService implements Sch
throw new org.apache.nifi.schema.access.SchemaNotFoundException("Cannot retrieve schema because Schema Name is not present");
}
return schemaNameMap.get(schemaName);
return schemaNameMap.get(schemaName.get());
}
private RecordSchema retrieveSchemaByIdAndVersion(final SchemaIdentifier schemaIdentifier) throws IOException, SchemaNotFoundException {

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.apache.nifi.serialization.record
import org.apache.avro.Schema
import org.apache.nifi.avro.AvroTypeUtil
import org.junit.Assert
import org.junit.Test
import static groovy.json.JsonOutput.*
class TestMockSchemaRegistry {
@Test
void testGetSchemaByName() {
def registry = new MockSchemaRegistry()
def schema = prettyPrint(toJson([
name: "TestSchema",
type: "record",
fields: [
[ name: "msg", type: "string" ]
]
]))
def recordSchema = AvroTypeUtil.createSchema(new Schema.Parser().parse(schema))
registry.addSchema("simple", recordSchema)
def identifier = SchemaIdentifier.builder().name("simple").build()
def result = registry.retrieveSchemaByName(identifier)
Assert.assertNotNull("Failed to load schema.", result)
Assert.assertEquals(result.fieldNames, recordSchema.fieldNames)
}
}