Uploaded image for project: 'Apache NiFi'
  1. Apache NiFi
  2. NIFI-5741

JndiJmsConnectionFactory does not properly wrap returned ConnectionFactory with ClassLoader-specific proxy

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.8.0
    • 1.9.0
    • Extensions
    • None

    Description

      The JndiJmsConnectionFactory controller service looks up a JMS Connection Factory via JNDI. It then returns this Connection Factory to be used by the JMS Processors. While the JMS Processors live in the same NAR, the instance of the JMS Processor will live have a different NAR Context Class Loader if the JNDI JMS Connection Factory is configured with classes that modify its class path.

      As a result, the JMS Processors can have the wrong class path when calling ConnectionFactory#createConnection, which can cause some very odd behavior. When attempting to connect to WebLogic, for instance, I got the following stack trace:

      2018-10-23 11:53:53,803 ERROR [Timer-Driven Process Thread-6] o.apache.nifi.jms.processors.PublishJMS PublishJMS[id=a191db80-0166-1000-77ff-aaaf7616fcb1] Failed while sending message to JMS via JMSPublisher[destination:null; pub-sub:true;]: org.springframework.jms.UncategorizedJmsException: Uncategorized exception occurred during JMS processing; nested exception is weblogic.jms.common.JMSException: [JMSClientExceptions:055053]Error creating connection to the server: java.rmi.UnmarshalException: failed to unmarshal class weblogic.jms.client.JMSConnection; nested exception is:
      {{ java.io.UTFDataFormatException.}}
      java.rmi.UnmarshalException: failed to unmarshal class weblogic.jms.client.JMSConnection; nested exception is:
      {{ java.io.UTFDataFormatException}}
      {{ at weblogic.rjvm.ResponseImpl.unmarshalReturn(ResponseImpl.java:266)}}
      {{ at weblogic.rmi.cluster.ClusterableRemoteRef.invoke(ClusterableRemoteRef.java:477)}}
      {{ at weblogic.rmi.cluster.ClusterableRemoteRef.invoke(ClusterableRemoteRef.java:285)}}
      {{ at weblogic.jms.frontend.FEConnectionFactoryImpl_12213_WLStub.connectionCreate(Unknown Source)}}
      {{ at weblogic.jms.client.JMSConnectionFactory.setupJMSConnection(JMSConnectionFactory.java:269)}}
      {{ at weblogic.jms.client.JMSConnectionFactory.createConnectionInternal(JMSConnectionFactory.java:359)}}
      {{ at weblogic.jms.client.JMSConnectionFactory.createConnection(JMSConnectionFactory.java:210)}}
      {{ at org.springframework.jms.connection.UserCredentialsConnectionFactoryAdapter.doCreateConnection(UserCredentialsConnectionFactoryAdapter.java:181)}}
      {{ at org.springframework.jms.connection.UserCredentialsConnectionFactoryAdapter.createConnection(UserCredentialsConnectionFactoryAdapter.java:152)}}
      {{ at org.springframework.jms.connection.SingleConnectionFactory.doCreateConnection(SingleConnectionFactory.java:366)}}
      {{ at org.springframework.jms.connection.SingleConnectionFactory.initConnection(SingleConnectionFactory.java:306)}}
      {{ at org.springframework.jms.connection.SingleConnectionFactory.getConnection(SingleConnectionFactory.java:284)}}
      {{ at org.springframework.jms.connection.SingleConnectionFactory.createConnection(SingleConnectionFactory.java:225)}}
      {{ at org.springframework.jms.support.JmsAccessor.createConnection(JmsAccessor.java:180)}}
      {{ at org.springframework.jms.core.JmsTemplate.execute(JmsTemplate.java:474)}}
      {{ at org.springframework.jms.core.JmsTemplate.send(JmsTemplate.java:570)}}
      {{ at org.apache.nifi.jms.processors.JMSPublisher.publish(JMSPublisher.java:56)}}
      {{ at org.apache.nifi.jms.processors.PublishJMS.rendezvousWithJms(PublishJMS.java:131)}}
      {{ at org.apache.nifi.jms.processors.PublishJMS.rendezvousWithJms(PublishJMS.java:60)}}
      {{ at org.apache.nifi.jms.processors.AbstractJMSProcessor.onTrigger(AbstractJMSProcessor.java:159)}}
      {{ at org.apache.nifi.jms.processors.PublishJMS.onTrigger(PublishJMS.java:60)}}
      {{ at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)}}
      {{ at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1165)}}
      {{ at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:203)}}
      {{ at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117)}}
      {{ 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:1142)}}
      {{ at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)}}
      {{ at java.lang.Thread.run(Thread.java:745)}}
      Caused by: java.io.UTFDataFormatException
      {{ at java.io.ObjectInputStream$BlockDataInputStream.readUTFSpan(ObjectInputStream.java:3137)}}
      {{ at java.io.ObjectInputStream$BlockDataInputStream.readUTFBody(ObjectInputStream.java:3062)}}
      {{ at java.io.ObjectInputStream$BlockDataInputStream.readUTF(ObjectInputStream.java:2874)}}
      {{ at java.io.ObjectInputStream.readUTF(ObjectInputStream.java:1073)}}
      {{ at weblogic.messaging.dispatcher.DispatcherUtils.readVersionedPartitionInfo(DispatcherUtils.java:185)}}
      {{ at weblogic.messaging.dispatcher.DispatcherWrapper.readExternal(DispatcherWrapper.java:236)}}
      {{ at weblogic.jms.client.JMSConnection.readExternal(JMSConnection.java:2481)}}
      {{ at java.io.ObjectInputStream.readExternalData(ObjectInputStream.java:1842)}}
      {{ at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1799)}}
      {{ at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)}}
      {{ at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)}}
      {{ at weblogic.utils.io.ChunkedObjectInputStream.readObject(ChunkedObjectInputStream.java:208)}}
      {{ at weblogic.rjvm.MsgAbbrevInputStream.readObject(MsgAbbrevInputStream.java:602)}}
      {{ at weblogic.utils.io.ChunkedObjectInputStream.readObject(ChunkedObjectInputStream.java:204)}}
      {{ at weblogic.rmi.internal.ObjectIO.readObject(ObjectIO.java:62)}}
      {{ at weblogic.rjvm.ResponseImpl.unmarshalReturn(ResponseImpl.java:264)}}
      {{ ... 31 more}}

       

      Attachments

        Issue Links

          Activity

            People

              markap14 Mark Payne
              markap14 Mark Payne
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: