That seemed to have fixed it. Thank you!

 

From: Mark Payne <marka...@hotmail.com> 
Sent: Thursday, October 22, 2020 11:54
To: users@nifi.apache.org
Subject: Re: NiFi 1.11.4 HDFS/HBASE Processor Errors After Kerberos Ticket 
Expires

 

This line in your bootstrap.conf appears to be the problem:

 

java.arg.16=-Djavax.security.auth.useSubjectCredsOnly=false

 

This needs to be changed to true:

java.arg.16=-Djavax.security.auth.useSubjectCredsOnly=true

 

My understanding is that with that set to false, you have a situation where 
Kerberos will prompt the user for a password. Since NiFi is running in the 
background, there’s no user to provide a password, so it will block waiting for 
that input that it never receives.





On Oct 22, 2020, at 11:43 AM, jw4306...@gmail.com <mailto:jw4306...@gmail.com>  
wrote:

 

There were two exception stack traces that occurred once we tried to have the 
processors do something after the ticket had expired:

 

2020-10-20 22:46:25,406 WARN [NiFi Web Server-30101] 
o.a.n.c.r.StandardProcessSession StandardProcessSession[id=5424590257] closing 
ErrorHandlingInputStream[FlowFile=StandardFlowFileRecord[uuid=e3d64666-bc6d-46fd-aacd-c20bbde05e10,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1603225642516-74, container=default, 
section=74], offset=644907, 
length=235],offset=0,name=3a05a988-b933-43f1-a6c5-8a39aa31400e,size=235]] for 
StandardFlowFileRecord[uuid=e3d64666-bc6d-46fd-aacd-c20bbde05e10,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1603225642516-74, container=default, 
section=74], offset=644907, 
length=235],offset=0,name=3a05a988-b933-43f1-a6c5-8a39aa31400e,size=235] 
because the session was rolled back without the input stream being closed.

 

 

2020-10-20 22:48:31,729 ERROR [Timer-Driven Process Thread-97 <Terminated 
Task>] org.apache.nifi.hbase.PutHBaseRecord 
PutHBaseRecord[id=3c00175e-2825-1445-fe2e-e0b37458ba34] Failed to put records 
to HBase.: java.io.InterruptedIOException: #6403, interrupted. 
currentNumberOfTask=1
java.io.InterruptedIOException: #6403, interrupted. currentNumberOfTask=1
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForMaximumCurrentTasks(AsyncProcess.java:1693)
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForMaximumCurrentTasks(AsyncProcess.java:1663)
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForAllPreviousOpsAndReset(AsyncProcess.java:1739)
at 
org.apache.hadoop.hbase.client.BufferedMutatorImpl.backgroundFlushCommits(BufferedMutatorImpl.java:208)
at 
org.apache.hadoop.hbase.client.BufferedMutatorImpl.flush(BufferedMutatorImpl.java:183)
at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1434)
at org.apache.hadoop.hbase.client.HTable.put(HTable.java:1030)
at 
org.apache.nifi.hbase.HBase_1_1_2_ClientService.put(HBase_1_1_2_ClientService.java:466)
at sun.reflect.GeneratedMethodAccessor187.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:87)
at com.sun.proxy.$Proxy121.put(Unknown Source)
at org.apache.nifi.hbase.PutHBaseRecord.addBatch(PutHBaseRecord.java:204)
at org.apache.nifi.hbase.PutHBaseRecord.onTrigger(PutHBaseRecord.java:277)
at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1176)
at 
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:213)
at 
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
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)
Suppressed: java.io.InterruptedIOException: #6403, interrupted. 
currentNumberOfTask=1
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForMaximumCurrentTasks(AsyncProcess.java:1693)
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForMaximumCurrentTasks(AsyncProcess.java:1663)
at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForAllPreviousOpsAndReset(AsyncProcess.java:1739)
at 
org.apache.hadoop.hbase.client.BufferedMutatorImpl.backgroundFlushCommits(BufferedMutatorImpl.java:208)
at 
org.apache.hadoop.hbase.client.BufferedMutatorImpl.flush(BufferedMutatorImpl.java:183)
at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1434)
at org.apache.hadoop.hbase.client.HTable.close(HTable.java:1470)
at 
org.apache.nifi.hbase.HBase_1_1_2_ClientService.put(HBase_1_1_2_ClientService.java:467)
... 19 common frames omitted
Suppressed: org.apache.nifi.processor.exception.TerminatedTaskException: null
at 
org.apache.nifi.controller.repository.io.TaskTerminationInputStream.verifyNotTerminated(TaskTerminationInputStream.java:39)
at 
org.apache.nifi.controller.repository.io.TaskTerminationInputStream.close(TaskTerminationInputStream.java:122)
at org.apache.commons.io.input.ProxyInputStream.close(ProxyInputStream.java:146)
at sun.nio.cs.StreamDecoder.implClose(StreamDecoder.java:378)
at sun.nio.cs.StreamDecoder.close(StreamDecoder.java:193)
at java.io.InputStreamReader.close(InputStreamReader.java:199)
at java.io.BufferedReader.close(BufferedReader.java:525)
at 
org.apache.commons.csv.ExtendedBufferedReader.close(ExtendedBufferedReader.java:188)
at org.apache.commons.csv.Lexer.close(Lexer.java:459)
at org.apache.commons.csv.CSVParser.close(CSVParser.java:383)
at org.apache.nifi.csv.CSVRecordReader.close(CSVRecordReader.java:154)
at org.apache.nifi.hbase.PutHBaseRecord.onTrigger(PutHBaseRecord.java:280)
... 12 common frames omitted
Suppressed: org.apache.nifi.processor.exception.TerminatedTaskException: null
at 
org.apache.nifi.controller.repository.io.TaskTerminationInputStream.verifyNotTerminated(TaskTerminationInputStream.java:39)
at 
org.apache.nifi.controller.repository.io.TaskTerminationInputStream.close(TaskTerminationInputStream.java:122)
at org.apache.nifi.hbase.PutHBaseRecord.onTrigger(PutHBaseRecord.java:280)
... 12 common frames omitted

 

 

Mr. Witt, what setting are you referring to? My current bootstrap.conf file is:

 

java=java

run.as=nifi.svc

lib.dir=./lib

conf.dir=/drrs/programs/nifi/conf

graceful.shutdown.seconds=20

java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true

java.arg.2=-Xms40g

java.arg.3=-Xmx40g

java.arg.4=-Djava.net.preferIPv4Stack=true

java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true

java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol

# Root key in hexadecimal format for encrypted sensitive configuration values

java.arg.14=-Djava.awt.headless=true

nifi.bootstrap.sensitive.key=<removed>

java.arg.15=-Djava.security.egd=file:/dev/urandom

java.arg.16=-Djavax.security.auth.useSubjectCredsOnly=false

java.arg.17=-Dzookeeper.admin.enableServer=false

notification.services.file=/drrs/programs/nifi/conf/bootstrap-notification-services.xml

notification.max.attempts=5

java.arg.18=-Dzookeeper.sasl.client.username=zookeeper.svc

java.arg.19=-Djava.security.auth.login.config=/drrs/programs/nifi/conf/nifi-jaas.conf

java.arg.curator.supress.excessive.logs=-Dcurator-log-only-first-connection-issue-as-error-level=true

java.arg.20=-Dsun.security.krb5.debug=true

java.arg.21=-Dsun.security.spnego.debug=true

java.arg.22=-Dsun.security.krb5.disableReferrals=true

nifi.start.notification.services=email-notification

nifi.stop.notification.services=email-notification

nifi.died.notification.services=email-notification

 

From: Joe Witt <joe.w...@gmail.com <mailto:joe.w...@gmail.com> > 
Sent: Wednesday, October 21, 2020 18:40
To: users@nifi.apache.org <mailto:users@nifi.apache.org> 
Subject: Re: NiFi 1.11.4 HDFS/HBASE Processor Errors After Kerberos Ticket 
Expires

 

If there is nothing in the logs but they stop working I suspect the issue is 
related the default prompt for name.  Update settings in bootstrap is most 
likely needed.

 

Thanks

 

On Wed, Oct 21, 2020 at 3:36 PM Peter Turcsanyi <turcsa...@apache.org 
<mailto:turcsa...@apache.org> > wrote:

Are there any exception stack traces in the log when the processors fail / 
before that?

 

On Thu, Oct 22, 2020 at 12:28 AM <jw4306...@gmail.com 
<mailto:jw4306...@gmail.com> > wrote:

Hello!

 

We’re running into a problem with NiFi 1.11.4.

 

Our HBASE/HDFS/Parquet processors are all configured with a master 
KeytabCredentialsService that is pointing to a Kerberos principal and keytab 
file. 

 

The environment’s /etc/krb5.conf file has the line renew_lifetime = 7d 
commented out due to an issue with Java-OpenJDK (that is apparently fixed but 
still shows up) causing “MESSAGE STREAM MODIFIED (41)” errors to appear 
whenever we have it uncommented.

 

When NiFi starts, it is able to kinit with the Kerberos KDC and is issued a 24 
hour ticket. Everything works fine right up until that ticket expires. Once the 
ticket expires, all of our HDFS/HBASE/Parquet processors start failing. 

 

I haven’t been able to find anything in our logs around the timeframe, but I 
can’t turn on debug logging for this because the logs are tremendously large 
when we do that (approximately 100-200 MB per minute and the problem only 
occurs at the 24 hour mark). 

 

How would we go about troubleshooting this issue?

 

Environment:

Red Hat Enterprise Linux 7.9

Apache NiFi 1.11.4

java-11-openjdk 11.0.8.10-1.el7

 

Thanks! 

 

Reply via email to