You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by Mike Thomsen <mi...@gmail.com> on 2018/03/26 13:52:41 UTC

Weird issue w/ Redis pool service in an integration test

Over the weekend I started playing around with a new processor called
PutRedisHash based on a request from the user list. I set up a really
simple IT and hit a problem pretty quickly. This passes validation:

@Test
public void testStandalone() throws Exception {
    final String attrName = "key.name";
    final String attrValue = "simple_test";
    RedisConnectionPool connectionPool = new RedisConnectionPoolService();
    TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);

    runner.addControllerService("connPool", connectionPool);
    runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
"localhost:6379");
    runner.enableControllerService(connectionPool);
    runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
    runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
    runner.assertValid();
}

As soon as I enqueue some data and call run(), I see the following
exception get thrown in the processor. I checked the Connection String
property, and it is marked as supporting EL and does call
evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
method.

java.lang.IllegalStateException: Attempting to Evaluate Expressions but
PropertyDescriptor[Connection String] indicates that the Expression
Language is not supported. If you realize that this is the case and do not
want this error to occur, it can be disabled by calling
TestRunner.setValidateExpressionUsage(false)
at
org.apache.nifi.util.MockPropertyValue.markEvaluated(MockPropertyValue.java:133)
at
org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:183)
at
org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:177)
at
org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:142)
at
org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(RedisUtils.java:260)
at
org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(AbstractRedisProcessor.java:41)
at
org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(AbstractRedisProcessor.java:50)
at
org.apache.nifi.processors.redis.PutRedisHash.onTrigger(PutRedisHash.java:162)
at
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at
org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:251)
at
org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:245)
at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
at java.util.concurrent.FutureTask.run(FutureTask.java)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)

Any ideas?

Re: Weird issue w/ Redis pool service in an integration test

Posted by Bryan Bende <bb...@gmail.com>.
Ah yea, take a look at RedisDistributedMapCacheClientService, that one
is closer to what you need.

Something like....

RedisConnectionPool redisConnectionPool =
context.getProperty(REDIS_CONNECTION_POOL).asControllerService(RedisConnectionPool.class);
RedisConnection redisConnection = redisConnectionPool.getConnection();



On Mon, Mar 26, 2018 at 11:58 AM, Mike Thomsen <mi...@gmail.com> wrote:
> Yeah, it does. Copied withConnection from the state provider. Looks like
> copya pasta may have struck again...
>
> On Mon, Mar 26, 2018 at 11:44 AM, Bryan Bende <bb...@gmail.com> wrote:
>
>> I can't tell for sure, but the stacktrace looks like your
>> AbstractRedisProcessor is making a direct call to RedisUtils to create
>> a connection, rather than using the RedisConnectionPool to obtain a
>> connection.
>>
>> On Mon, Mar 26, 2018 at 11:38 AM, Bryan Bende <bb...@gmail.com> wrote:
>> > Can you share the code for your AbstractRedisProcessor?
>> >
>> >
>> > On Mon, Mar 26, 2018 at 9:52 AM, Mike Thomsen <mi...@gmail.com>
>> wrote:
>> >> Over the weekend I started playing around with a new processor called
>> >> PutRedisHash based on a request from the user list. I set up a really
>> >> simple IT and hit a problem pretty quickly. This passes validation:
>> >>
>> >> @Test
>> >> public void testStandalone() throws Exception {
>> >>     final String attrName = "key.name";
>> >>     final String attrValue = "simple_test";
>> >>     RedisConnectionPool connectionPool = new
>> RedisConnectionPoolService();
>> >>     TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);
>> >>
>> >>     runner.addControllerService("connPool", connectionPool);
>> >>     runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
>> >> "localhost:6379");
>> >>     runner.enableControllerService(connectionPool);
>> >>     runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
>> >>     runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
>> >>     runner.assertValid();
>> >> }
>> >>
>> >> As soon as I enqueue some data and call run(), I see the following
>> >> exception get thrown in the processor. I checked the Connection String
>> >> property, and it is marked as supporting EL and does call
>> >> evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
>> >> method.
>> >>
>> >> java.lang.IllegalStateException: Attempting to Evaluate Expressions but
>> >> PropertyDescriptor[Connection String] indicates that the Expression
>> >> Language is not supported. If you realize that this is the case and do
>> not
>> >> want this error to occur, it can be disabled by calling
>> >> TestRunner.setValidateExpressionUsage(false)
>> >> at
>> >> org.apache.nifi.util.MockPropertyValue.markEvaluated(
>> MockPropertyValue.java:133)
>> >> at
>> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
>> MockPropertyValue.java:183)
>> >> at
>> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
>> MockPropertyValue.java:177)
>> >> at
>> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
>> MockPropertyValue.java:142)
>> >> at
>> >> org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(
>> RedisUtils.java:260)
>> >> at
>> >> org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(
>> AbstractRedisProcessor.java:41)
>> >> at
>> >> org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(
>> AbstractRedisProcessor.java:50)
>> >> at
>> >> org.apache.nifi.processors.redis.PutRedisHash.onTrigger(
>> PutRedisHash.java:162)
>> >> at
>> >> org.apache.nifi.processor.AbstractProcessor.onTrigger(
>> AbstractProcessor.java:27)
>> >> at
>> >> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
>> StandardProcessorTestRunner.java:251)
>> >> at
>> >> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
>> StandardProcessorTestRunner.java:245)
>> >> at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
>> >> at java.util.concurrent.FutureTask.run(FutureTask.java)
>> >> at
>> >> java.util.concurrent.ScheduledThreadPoolExecutor$
>> ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> >> at
>> >> java.util.concurrent.ScheduledThreadPoolExecutor$
>> ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> >> at
>> >> java.util.concurrent.ThreadPoolExecutor.runWorker(
>> ThreadPoolExecutor.java:1149)
>> >> at
>> >> java.util.concurrent.ThreadPoolExecutor$Worker.run(
>> ThreadPoolExecutor.java:624)
>> >> at java.lang.Thread.run(Thread.java:748)
>> >>
>> >> Any ideas?
>>

Re: Weird issue w/ Redis pool service in an integration test

Posted by Mike Thomsen <mi...@gmail.com>.
Yeah, it does. Copied withConnection from the state provider. Looks like
copya pasta may have struck again...

On Mon, Mar 26, 2018 at 11:44 AM, Bryan Bende <bb...@gmail.com> wrote:

> I can't tell for sure, but the stacktrace looks like your
> AbstractRedisProcessor is making a direct call to RedisUtils to create
> a connection, rather than using the RedisConnectionPool to obtain a
> connection.
>
> On Mon, Mar 26, 2018 at 11:38 AM, Bryan Bende <bb...@gmail.com> wrote:
> > Can you share the code for your AbstractRedisProcessor?
> >
> >
> > On Mon, Mar 26, 2018 at 9:52 AM, Mike Thomsen <mi...@gmail.com>
> wrote:
> >> Over the weekend I started playing around with a new processor called
> >> PutRedisHash based on a request from the user list. I set up a really
> >> simple IT and hit a problem pretty quickly. This passes validation:
> >>
> >> @Test
> >> public void testStandalone() throws Exception {
> >>     final String attrName = "key.name";
> >>     final String attrValue = "simple_test";
> >>     RedisConnectionPool connectionPool = new
> RedisConnectionPoolService();
> >>     TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);
> >>
> >>     runner.addControllerService("connPool", connectionPool);
> >>     runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
> >> "localhost:6379");
> >>     runner.enableControllerService(connectionPool);
> >>     runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
> >>     runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
> >>     runner.assertValid();
> >> }
> >>
> >> As soon as I enqueue some data and call run(), I see the following
> >> exception get thrown in the processor. I checked the Connection String
> >> property, and it is marked as supporting EL and does call
> >> evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
> >> method.
> >>
> >> java.lang.IllegalStateException: Attempting to Evaluate Expressions but
> >> PropertyDescriptor[Connection String] indicates that the Expression
> >> Language is not supported. If you realize that this is the case and do
> not
> >> want this error to occur, it can be disabled by calling
> >> TestRunner.setValidateExpressionUsage(false)
> >> at
> >> org.apache.nifi.util.MockPropertyValue.markEvaluated(
> MockPropertyValue.java:133)
> >> at
> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:183)
> >> at
> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:177)
> >> at
> >> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:142)
> >> at
> >> org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(
> RedisUtils.java:260)
> >> at
> >> org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(
> AbstractRedisProcessor.java:41)
> >> at
> >> org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(
> AbstractRedisProcessor.java:50)
> >> at
> >> org.apache.nifi.processors.redis.PutRedisHash.onTrigger(
> PutRedisHash.java:162)
> >> at
> >> org.apache.nifi.processor.AbstractProcessor.onTrigger(
> AbstractProcessor.java:27)
> >> at
> >> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
> StandardProcessorTestRunner.java:251)
> >> at
> >> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
> StandardProcessorTestRunner.java:245)
> >> at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
> >> at java.util.concurrent.FutureTask.run(FutureTask.java)
> >> at
> >> java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> >> at
> >> java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> >> at
> >> java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1149)
> >> at
> >> java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:624)
> >> at java.lang.Thread.run(Thread.java:748)
> >>
> >> Any ideas?
>

Re: Weird issue w/ Redis pool service in an integration test

Posted by Bryan Bende <bb...@gmail.com>.
I can't tell for sure, but the stacktrace looks like your
AbstractRedisProcessor is making a direct call to RedisUtils to create
a connection, rather than using the RedisConnectionPool to obtain a
connection.

On Mon, Mar 26, 2018 at 11:38 AM, Bryan Bende <bb...@gmail.com> wrote:
> Can you share the code for your AbstractRedisProcessor?
>
>
> On Mon, Mar 26, 2018 at 9:52 AM, Mike Thomsen <mi...@gmail.com> wrote:
>> Over the weekend I started playing around with a new processor called
>> PutRedisHash based on a request from the user list. I set up a really
>> simple IT and hit a problem pretty quickly. This passes validation:
>>
>> @Test
>> public void testStandalone() throws Exception {
>>     final String attrName = "key.name";
>>     final String attrValue = "simple_test";
>>     RedisConnectionPool connectionPool = new RedisConnectionPoolService();
>>     TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);
>>
>>     runner.addControllerService("connPool", connectionPool);
>>     runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
>> "localhost:6379");
>>     runner.enableControllerService(connectionPool);
>>     runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
>>     runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
>>     runner.assertValid();
>> }
>>
>> As soon as I enqueue some data and call run(), I see the following
>> exception get thrown in the processor. I checked the Connection String
>> property, and it is marked as supporting EL and does call
>> evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
>> method.
>>
>> java.lang.IllegalStateException: Attempting to Evaluate Expressions but
>> PropertyDescriptor[Connection String] indicates that the Expression
>> Language is not supported. If you realize that this is the case and do not
>> want this error to occur, it can be disabled by calling
>> TestRunner.setValidateExpressionUsage(false)
>> at
>> org.apache.nifi.util.MockPropertyValue.markEvaluated(MockPropertyValue.java:133)
>> at
>> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:183)
>> at
>> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:177)
>> at
>> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:142)
>> at
>> org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(RedisUtils.java:260)
>> at
>> org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(AbstractRedisProcessor.java:41)
>> at
>> org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(AbstractRedisProcessor.java:50)
>> at
>> org.apache.nifi.processors.redis.PutRedisHash.onTrigger(PutRedisHash.java:162)
>> at
>> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
>> at
>> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:251)
>> at
>> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:245)
>> at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
>> at java.util.concurrent.FutureTask.run(FutureTask.java)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>> at java.lang.Thread.run(Thread.java:748)
>>
>> Any ideas?

Re: Weird issue w/ Redis pool service in an integration test

Posted by Mike Thomsen <mi...@gmail.com>.
https://github.com/MikeThomsen/nifi/commit/dad71af803a9ead5bcbead3ce65671c4c2430605#diff-09d6acf9b469f7d2491ab7dbdab93fea

On Mon, Mar 26, 2018 at 11:38 AM, Bryan Bende <bb...@gmail.com> wrote:

> Can you share the code for your AbstractRedisProcessor?
>
>
> On Mon, Mar 26, 2018 at 9:52 AM, Mike Thomsen <mi...@gmail.com>
> wrote:
> > Over the weekend I started playing around with a new processor called
> > PutRedisHash based on a request from the user list. I set up a really
> > simple IT and hit a problem pretty quickly. This passes validation:
> >
> > @Test
> > public void testStandalone() throws Exception {
> >     final String attrName = "key.name";
> >     final String attrValue = "simple_test";
> >     RedisConnectionPool connectionPool = new
> RedisConnectionPoolService();
> >     TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);
> >
> >     runner.addControllerService("connPool", connectionPool);
> >     runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
> > "localhost:6379");
> >     runner.enableControllerService(connectionPool);
> >     runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
> >     runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
> >     runner.assertValid();
> > }
> >
> > As soon as I enqueue some data and call run(), I see the following
> > exception get thrown in the processor. I checked the Connection String
> > property, and it is marked as supporting EL and does call
> > evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
> > method.
> >
> > java.lang.IllegalStateException: Attempting to Evaluate Expressions but
> > PropertyDescriptor[Connection String] indicates that the Expression
> > Language is not supported. If you realize that this is the case and do
> not
> > want this error to occur, it can be disabled by calling
> > TestRunner.setValidateExpressionUsage(false)
> > at
> > org.apache.nifi.util.MockPropertyValue.markEvaluated(
> MockPropertyValue.java:133)
> > at
> > org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:183)
> > at
> > org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:177)
> > at
> > org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(
> MockPropertyValue.java:142)
> > at
> > org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(
> RedisUtils.java:260)
> > at
> > org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(
> AbstractRedisProcessor.java:41)
> > at
> > org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(
> AbstractRedisProcessor.java:50)
> > at
> > org.apache.nifi.processors.redis.PutRedisHash.onTrigger(
> PutRedisHash.java:162)
> > at
> > org.apache.nifi.processor.AbstractProcessor.onTrigger(
> AbstractProcessor.java:27)
> > at
> > org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
> StandardProcessorTestRunner.java:251)
> > at
> > org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(
> StandardProcessorTestRunner.java:245)
> > at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
> > at java.util.concurrent.FutureTask.run(FutureTask.java)
> > at
> > java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> > at
> > java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> > at
> > java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1149)
> > at
> > java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:624)
> > at java.lang.Thread.run(Thread.java:748)
> >
> > Any ideas?
>

Re: Weird issue w/ Redis pool service in an integration test

Posted by Bryan Bende <bb...@gmail.com>.
Can you share the code for your AbstractRedisProcessor?


On Mon, Mar 26, 2018 at 9:52 AM, Mike Thomsen <mi...@gmail.com> wrote:
> Over the weekend I started playing around with a new processor called
> PutRedisHash based on a request from the user list. I set up a really
> simple IT and hit a problem pretty quickly. This passes validation:
>
> @Test
> public void testStandalone() throws Exception {
>     final String attrName = "key.name";
>     final String attrValue = "simple_test";
>     RedisConnectionPool connectionPool = new RedisConnectionPoolService();
>     TestRunner runner = TestRunners.newTestRunner(PutRedisHash.class);
>
>     runner.addControllerService("connPool", connectionPool);
>     runner.setProperty(connectionPool, RedisUtils.CONNECTION_STRING,
> "localhost:6379");
>     runner.enableControllerService(connectionPool);
>     runner.setProperty(PutRedisHash.REDIS_CONNECTION_POOL, "connPool");
>     runner.setProperty(PutRedisHash.NAME_ATTRIBUTE, attrName);
>     runner.assertValid();
> }
>
> As soon as I enqueue some data and call run(), I see the following
> exception get thrown in the processor. I checked the Connection String
> property, and it is marked as supporting EL and does call
> evaluationExpressionLanguage in the RedisUtils.createConnectionFactory
> method.
>
> java.lang.IllegalStateException: Attempting to Evaluate Expressions but
> PropertyDescriptor[Connection String] indicates that the Expression
> Language is not supported. If you realize that this is the case and do not
> want this error to occur, it can be disabled by calling
> TestRunner.setValidateExpressionUsage(false)
> at
> org.apache.nifi.util.MockPropertyValue.markEvaluated(MockPropertyValue.java:133)
> at
> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:183)
> at
> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:177)
> at
> org.apache.nifi.util.MockPropertyValue.evaluateAttributeExpressions(MockPropertyValue.java:142)
> at
> org.apache.nifi.redis.util.RedisUtils.createConnectionFactory(RedisUtils.java:260)
> at
> org.apache.nifi.processors.redis.AbstractRedisProcessor.getRedis(AbstractRedisProcessor.java:41)
> at
> org.apache.nifi.processors.redis.AbstractRedisProcessor.withConnection(AbstractRedisProcessor.java:50)
> at
> org.apache.nifi.processors.redis.PutRedisHash.onTrigger(PutRedisHash.java:162)
> at
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> at
> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:251)
> at
> org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:245)
> at java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:266)
> at java.util.concurrent.FutureTask.run(FutureTask.java)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
>
> Any ideas?