Skip to content

Commit

Permalink
[STORM-3043] Fix NullPointerException when apply() returns null
Browse files Browse the repository at this point in the history
  • Loading branch information
cleroux authored and srdo committed Apr 28, 2018
1 parent 22a9620 commit cfe1124
Show file tree
Hide file tree
Showing 3 changed files with 79 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,12 @@ public SimpleRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fie

@Override
public List<Object> apply(ConsumerRecord<K, V> record) {
List<Object> vals = func.apply(record);
if (vals == null) {
return null;
}
KafkaTuple ret = new KafkaTuple();
ret.addAll(func.apply(record));
ret.addAll(vals);
return ret.routedTo(stream);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,13 @@ public List<Object> apply(ConsumerRecord<String, String> record) {
return new Values(record.key(), record.value());
}
};

public static Func<ConsumerRecord<String, String>, List<Object>> NULL_FUNC = new Func<ConsumerRecord<String, String>, List<Object>>() {
@Override
public List<Object> apply(ConsumerRecord<String, String> record) {
return null;
}
};

@Test
public void testBasic() {
Expand All @@ -74,6 +81,14 @@ public void testBasic() {
assertEquals(new Fields("key", "value"), trans.getFieldsFor("key-value-stream"));
assertEquals(Arrays.asList("THE KEY", "THE VALUE"), trans.apply(cr3));
}

@Test
public void testNullTranslation() {
ByTopicRecordTranslator<String, String> trans =
new ByTopicRecordTranslator<>(NULL_FUNC, new Fields("key"));
ConsumerRecord<String, String> cr = new ConsumerRecord<>("TOPIC 1", 100, 100, "THE KEY", "THE VALUE");
assertEquals(null, trans.apply(cr));
}

@Test(expected = IllegalArgumentException.class)
public void testFieldCollision() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.storm.kafka.spout;

import static org.junit.Assert.assertEquals;

import java.util.Arrays;
import java.util.List;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.storm.tuple.Fields;
import org.apache.storm.tuple.Values;
import org.junit.Test;

public class SimpleRecordTranslatorTest {
public static Func<ConsumerRecord<String, String>, List<Object>> JUST_VALUE_FUNC = new Func<ConsumerRecord<String, String>, List<Object>>() {
@Override
public List<Object> apply(ConsumerRecord<String, String> record) {
return new Values(record.value());
}
};

public static Func<ConsumerRecord<String, String>, List<Object>> NULL_FUNC = new Func<ConsumerRecord<String, String>, List<Object>>() {
@Override
public List<Object> apply(ConsumerRecord<String, String> record) {
return null;
}
};

@Test
public void testBasic() {
SimpleRecordTranslator<String, String> trans =
new SimpleRecordTranslator<>(JUST_VALUE_FUNC, new Fields("value"));
assertEquals(Arrays.asList("default"), trans.streams());
ConsumerRecord<String, String> cr = new ConsumerRecord<>("TOPIC", 100, 100, "THE KEY", "THE VALUE");
assertEquals(Arrays.asList("THE VALUE"), trans.apply(cr));
}

@Test
public void testNullTranslation() {
SimpleRecordTranslator<String, String> trans =
new SimpleRecordTranslator<>(NULL_FUNC, new Fields("key"));
assertEquals(null, trans.apply(null));
}
}

0 comments on commit cfe1124

Please sign in to comment.