You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "markap14 (via GitHub)" <gi...@apache.org> on 2023/10/18 12:47:19 UTC

[PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

markap14 opened a new pull request, #7894:
URL: https://github.com/apache/nifi/pull/7894

   …ings, inserting into Chroma, querying Chroma, querying ChatGPT, inserting into and querying Pinecone. Fixed some bugs in the Python framework. Added Python extensions to assembly. Also added ability to load dependencies from a requirements.txt as that was important for making the different vectorstore implementations play more nicely together.
   
   <!-- 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. -->
   
   # Summary
   
   [NIFI-00000](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 21
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1374842776


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/ParseUnstructuredDocument.py:
##########
@@ -0,0 +1,259 @@
+# 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.
+
+import io
+import json
+from typing import List
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency
+
+PLAIN_TEXT = "Plain Text"
+HTML = "HTML"
+MARKDOWN = "Markdown"
+PDF = "PDF"
+EXCEL = "Microsoft Excel"
+POWERPOINT = "Microsoft PowerPoint"
+WORD = "Microsoft Word"
+
+PARSING_STRATEGY_AUTO = "Automatic"
+PARSING_STRATEGY_HIGH_RES = "High Resolution"
+PARSING_STRATEGY_OCR_ONLY = "OCR Only"
+PARSING_STRATEGY_FAST = "Fast"
+
+SINGLE_DOCUMENT = "Single Document"
+DOCUMENT_PER_ELEMENT = "Document Per Element"
+
+TEXT_KEY = "text"
+METADATA_KEY = "metadata"
+
+
+class ParseUnstructuredDocument(FlowFileTransform):

Review Comment:
   would ParseDocument be a better name?  It doesn't seem critical to think of it as structured or unstructured and makes the name quite long.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1374906530


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/ParseUnstructuredDocument.py:
##########
@@ -0,0 +1,259 @@
+# 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.
+
+import io
+import json
+from typing import List
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency
+
+PLAIN_TEXT = "Plain Text"
+HTML = "HTML"
+MARKDOWN = "Markdown"
+PDF = "PDF"
+EXCEL = "Microsoft Excel"
+POWERPOINT = "Microsoft PowerPoint"
+WORD = "Microsoft Word"
+
+PARSING_STRATEGY_AUTO = "Automatic"
+PARSING_STRATEGY_HIGH_RES = "High Resolution"
+PARSING_STRATEGY_OCR_ONLY = "OCR Only"
+PARSING_STRATEGY_FAST = "Fast"
+
+SINGLE_DOCUMENT = "Single Document"
+DOCUMENT_PER_ELEMENT = "Document Per Element"
+
+TEXT_KEY = "text"
+METADATA_KEY = "metadata"
+
+
+class ParseUnstructuredDocument(FlowFileTransform):

Review Comment:
   Thanks @joewitt. I went back and forth on whether to have the word 'unstructured' there or not. The only concern that I had with 'ParseDocument' is that it might be vague. But if you think ParseDocument fits better, will change to that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377930956


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/PutChroma.py:
##########
@@ -0,0 +1,125 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import ChromaUtils
+import EmbeddingUtils
+
+
+class PutChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = """Publishes JSON data to a Chroma VectorDB. The Incoming data must be in single JSON per Line format, each with two keys: 'text' and 'metadata'.
+                       The text must be a string, while metadata must be a map with strings for values. Any additional fields will be ignored. If the collection name specified
+                       does not exist, the Processor will automatically create the collection."""
+        tags = ["chroma", "vector", "vectordb", "embeddings", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+    STORE_TEXT = PropertyDescriptor(
+        name="Store Document Text",
+        description="Specifies whether or not the text of the document should be stored in Chroma. If so, both the document's text and its embedding will be stored. If not, " +
+                    "only the vector/embedding will be stored.",
+        allowable_values=["true", "false"],
+        required=True,
+        default_value="true"
+    )
+    DISTANCE_METHOD = PropertyDescriptor(
+        name="Distance Method",
+        description="If the specified collection does not exist, it will be created using this Distance Method. If the collection exists, this property will be ignored.",
+        allowable_values=["cosine", "l2", "ip"],
+        default_value="cosine",
+        required=True
+    )
+    DOC_ID_FIELD_NAME = PropertyDescriptor(
+        name="Document ID Field Name",
+        description="Specifies the name of the field in the 'metadata' element of each document where the document's ID can be found. " +
+                    "If not specified, an ID will be generated based on the FlowFile's filename and a one-up number.",
+        required=False,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+
+
+    client = None
+    embedding_function = None
+
+    def __init__(self, **kwargs):
+        self.property_descriptors = [prop for prop in ChromaUtils.PROPERTIES] + [prop for prop in EmbeddingUtils.PROPERTIES]
+        self.property_descriptors.append(self.STORE_TEXT)
+        self.property_descriptors.append(self.DISTANCE_METHOD)
+        self.property_descriptors.append(self.DOC_ID_FIELD_NAME)
+
+
+    def getPropertyDescriptors(self):
+        return self.property_descriptors
+
+    def onScheduled(self, context):
+        self.client = ChromaUtils.create_client(context)
+        self.embedding_function = EmbeddingUtils.create_embedding_function(context)
+
+
+    def transform(self, context, flowfile):
+        client = self.client
+        embedding_function = self.embedding_function
+        collection_name = context.getProperty(ChromaUtils.COLLECTION_NAME).evaluateAttributeExpressions(flowfile).getValue()
+        distance_method = context.getProperty(self.DISTANCE_METHOD).getValue()
+        id_field_name = context.getProperty(self.DOC_ID_FIELD_NAME).evaluateAttributeExpressions(flowfile).getValue()
+
+        collection = client.get_or_create_collection(
+            name=collection_name,
+            embedding_function=embedding_function,
+            metadata={"hnsw:space": distance_method})
+
+        json_lines = flowfile.getContentsAsBytes().decode()

Review Comment:
   I'd say yes and no. In this space, we tend to see code that assumes that one document can fit well within the memory limits of the application. While in Java based processors, this would never fly, it feels a bit safer here, because of the fact that there is no shared heap. Unfortunately, if we were to use streaming, with the current architecture, it would require an RPC call per line of text, which would result in unacceptable performance. I think this is something that we can likely improve over time. For example, we might actually end up passing a reference to the content repository/offset/length and then make a `BytesIO` available to the Processor. But I didn't see a cleaner way to handle this in the meantime.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377942902


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/ChunkDocuments.py:
##########
@@ -0,0 +1,211 @@
+# 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.
+
+import json
+
+from langchain.text_splitter import Language
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency, ExpressionLanguageScope
+
+SPLIT_BY_CHARACTER = 'Split by Character'
+SPLIT_CODE = 'Split Code'
+RECURSIVELY_SPLIT_BY_CHARACTER = 'Recursively Split by Character'
+
+TEXT_KEY = "text"
+METADATA_KEY = "metadata"
+
+
+class ChunkDocuments(FlowFileTransform):

Review Comment:
   Agreed, good catch!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377921727


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",

Review Comment:
   It will always return this number of results. (Unless there are fewer than this number in the db).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1373830550


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryPinecone.py:
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from langchain.vectorstores import Pinecone
+from langchain.embeddings.openai import OpenAIEmbeddings
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import QueryUtils
+import pinecone
+
+
+class QueryPinecone(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries Pinecone in order to gather a specified number of documents that are most closely related to the given query."

Review Comment:
   This should actually have been done at the higher level, the Python Processor Proxy level - it should require input. Good catch. Will address that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "krisztina-zsihovszki (via GitHub)" <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1373753687


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryPinecone.py:
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from langchain.vectorstores import Pinecone
+from langchain.embeddings.openai import OpenAIEmbeddings
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import QueryUtils
+import pinecone
+
+
+class QueryPinecone(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries Pinecone in order to gather a specified number of documents that are most closely related to the given query."

Review Comment:
   In my view it'd be useful to mention that QueryPinecone, QueryChroma and PromptChatGPT require an incoming flowfile, otherwise the queries are not triggered (no output flow file is created, even if there was a match based on the processor properties). 



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/PutPinecone.py:
##########
@@ -0,0 +1,140 @@
+# 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.
+
+from langchain.vectorstores import Pinecone
+from langchain.embeddings.openai import OpenAIEmbeddings
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import pinecone
+import json
+
+
+class PutPinecone(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = """Publishes JSON data to a Pinecone. The Incoming data must be in single JSON per Line format, each with two keys: 'text' and 'metadata'.
+                       The text must be a string, while metadata must be a map with strings for values. Any additional fields will be ignored."""
+        tags = ["pinecone", "vector", "vectordb", "vectorstore", "embeddings", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+    PINECONE_API_KEY = PropertyDescriptor(
+        name="Pinecone API Key",
+        description="The API Key to use in order to authentication with Pinecone",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    OPENAI_API_KEY = PropertyDescriptor(
+        name="OpenAI API Key",
+        description="The API Key for OpenAI in order to create embeddings",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    PINECONE_ENV = PropertyDescriptor(
+        name="Pinecone Environment",
+        description="The name of the Pinecone Environment. This can be found in the Pinecone console next to the API Key.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    INDEX_NAME = PropertyDescriptor(
+        name="Index Name",
+        description="The name of the Pinecone index.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    TEXT_KEY = PropertyDescriptor(
+        name="Text Key",
+        description="The key in the document that contains the text to create embeddings for.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        default_value="text",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NAMESPACE = PropertyDescriptor(
+        name="Namespace",
+        description="The name of the Pinecone Namespace to put the documents to.",
+        required=False,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+
+    properties = [PINECONE_API_KEY,
+                  OPENAI_API_KEY,
+                  PINECONE_ENV,
+                  INDEX_NAME,
+                  TEXT_KEY,
+                  NAMESPACE]
+
+    embeddings = None
+
+    def __init__(self, **kwargs):
+        pass
+
+    def getPropertyDescriptors(self):
+        return self.properties
+
+    def onScheduled(self, context):
+        api_key = context.getProperty(self.PINECONE_API_KEY).getValue()
+        pinecone_env = context.getProperty(self.PINECONE_ENV).getValue()
+
+        # initialize pinecone
+        pinecone.init(
+            api_key=api_key,
+            environment=pinecone_env,
+        )
+        openai_api_key = context.getProperty(self.OPENAI_API_KEY).getValue()
+        self.embeddings = OpenAIEmbeddings(openai_api_key=openai_api_key)
+
+
+    def transform(self, context, flowfile):
+        # First, check if our index already exists. If it doesn't, we create it
+        index_name = context.getProperty(self.INDEX_NAME).evaluateAttributeExpressions(flowfile).getValue()
+        namespace = context.getProperty(self.NAMESPACE).evaluateAttributeExpressions(flowfile).getValue()
+
+        index = pinecone.Index(index_name)
+
+        # Read the FlowFile content as "json-lines".
+        json_lines = flowfile.getContentsAsBytes().decode()
+        i = 0
+        texts = []
+        metadatas = []
+        ids = []
+        for line in json_lines.split("\n"):
+            doc = json.loads(line)
+            text = doc.get('text')

Review Comment:
   The field name 'text' is used here while the text key can be overwritten by TEXT_KEY property.
   The processor description also mentions the 'text' as key, maybe it's worth mentioning that this is only the default key name.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/ChromaUtils.py:
##########
@@ -0,0 +1,155 @@
+# 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.
+
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency, ExpressionLanguageScope
+
+# Connection Strategies
+LOCAL_DISK = "Local Disk"
+REMOTE_SERVER = "Remote Chroma Server"
+
+# Authentication Strategies
+TOKEN = "Token Authentication"
+BASIC_AUTH = "Basic Authentication"
+NONE = "None"
+
+# Transport Protocols
+HTTP = "http"
+HTTPS = "https"
+
+CONNECTION_STRATEGY = PropertyDescriptor(
+    name="Connection Strategy",
+    description="Specifies how to connect to the Chroma server",
+    allowable_values=[LOCAL_DISK, REMOTE_SERVER],
+    default_value=REMOTE_SERVER,
+    required=True
+)
+DIRECTORY = PropertyDescriptor(
+    name="Directory",
+    description="The Directory that Chroma should use to persist data",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    default_value="./chroma",
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, LOCAL_DISK)]
+)
+HOSTNAME = PropertyDescriptor(
+    name="Hostname",
+    description="The hostname to connect to in order to communicate with Chroma",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    default_value="localhost",
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+PORT = PropertyDescriptor(
+    name="Port",
+    description="The port that the Chroma server is listening on",
+    validators=[StandardValidators.PORT_VALIDATOR],
+    default_value="8000",
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+TRANSPORT_PROTOCOL = PropertyDescriptor(
+    name="Transport Protocol",
+    description="Specifies whether connections should be made over http or https",
+    allowable_values=[HTTP, HTTPS],
+    default_value=HTTPS,
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+AUTH_STRATEGY = PropertyDescriptor(
+    name="Authentication Strategy",
+    description="Specifies how to authenticate to Chroma server",
+    allowable_values=[TOKEN, BASIC_AUTH, NONE],
+    default_value=TOKEN,
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+AUTH_TOKEN = PropertyDescriptor(
+    name="Authentication Token",
+    description="The token to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    sensitive=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, TOKEN)]
+)
+USERNAME = PropertyDescriptor(
+    name="Username",
+    description="The username to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, BASIC_AUTH)]
+)
+PASSWORD = PropertyDescriptor(
+    name="Password",
+    description="The password to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    sensitive=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, BASIC_AUTH)]
+)
+COLLECTION_NAME = PropertyDescriptor(
+    name="Collection Name",
+    description="The name of the Chroma Collection to update",

Review Comment:
   Minor: It'd be useful to mention here as well that the collection is created if it does not exist. 
   (It is mentioned only in the description of "Distance Method".)



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryPinecone.py:
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from langchain.vectorstores import Pinecone
+from langchain.embeddings.openai import OpenAIEmbeddings
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import QueryUtils
+import pinecone
+
+
+class QueryPinecone(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries Pinecone in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["pinecone", "vector", "vectordb", "vectorstore", "embeddings", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+    PINECONE_API_KEY = PropertyDescriptor(
+        name="Pinecone API Key",
+        description="The API Key to use in order to authentication with Pinecone",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    OPENAI_API_KEY = PropertyDescriptor(
+        name="OpenAI API Key",
+        description="The API Key for OpenAI in order to create embeddings",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    PINECONE_ENV = PropertyDescriptor(
+        name="Pinecone Environment",
+        description="The name of the Pinecone Environment. This can be found in the Pinecone console next to the API Key.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    INDEX_NAME = PropertyDescriptor(
+        name="Index Name",
+        description="The name of the Pinecone index.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to Pinecone.",

Review Comment:
   It is not obvious what kind of query is required here (it could have been a vectory query as well).
   It would be useful if something like this was added to the description: "Text to look up documents similar to." 
   or a similar description was used as used for QueryChroma's "Query" property.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryPinecone.py:
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from langchain.vectorstores import Pinecone
+from langchain.embeddings.openai import OpenAIEmbeddings
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import QueryUtils
+import pinecone
+
+
+class QueryPinecone(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries Pinecone in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["pinecone", "vector", "vectordb", "vectorstore", "embeddings", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+    PINECONE_API_KEY = PropertyDescriptor(
+        name="Pinecone API Key",
+        description="The API Key to use in order to authentication with Pinecone",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    OPENAI_API_KEY = PropertyDescriptor(
+        name="OpenAI API Key",
+        description="The API Key for OpenAI in order to create embeddings",
+        sensitive=True,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    PINECONE_ENV = PropertyDescriptor(
+        name="Pinecone Environment",
+        description="The name of the Pinecone Environment. This can be found in the Pinecone console next to the API Key.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR]
+    )
+    INDEX_NAME = PropertyDescriptor(
+        name="Index Name",
+        description="The name of the Pinecone index.",
+        sensitive=False,
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to Pinecone.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",

Review Comment:
   "...to return from Pinecone"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377875690


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/PutChroma.py:
##########
@@ -0,0 +1,125 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import ChromaUtils
+import EmbeddingUtils
+
+
+class PutChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = """Publishes JSON data to a Chroma VectorDB. The Incoming data must be in single JSON per Line format, each with two keys: 'text' and 'metadata'.
+                       The text must be a string, while metadata must be a map with strings for values. Any additional fields will be ignored. If the collection name specified
+                       does not exist, the Processor will automatically create the collection."""
+        tags = ["chroma", "vector", "vectordb", "embeddings", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+    STORE_TEXT = PropertyDescriptor(
+        name="Store Document Text",
+        description="Specifies whether or not the text of the document should be stored in Chroma. If so, both the document's text and its embedding will be stored. If not, " +
+                    "only the vector/embedding will be stored.",
+        allowable_values=["true", "false"],
+        required=True,
+        default_value="true"
+    )
+    DISTANCE_METHOD = PropertyDescriptor(
+        name="Distance Method",
+        description="If the specified collection does not exist, it will be created using this Distance Method. If the collection exists, this property will be ignored.",
+        allowable_values=["cosine", "l2", "ip"],
+        default_value="cosine",
+        required=True
+    )
+    DOC_ID_FIELD_NAME = PropertyDescriptor(
+        name="Document ID Field Name",
+        description="Specifies the name of the field in the 'metadata' element of each document where the document's ID can be found. " +
+                    "If not specified, an ID will be generated based on the FlowFile's filename and a one-up number.",
+        required=False,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+
+
+    client = None
+    embedding_function = None
+
+    def __init__(self, **kwargs):
+        self.property_descriptors = [prop for prop in ChromaUtils.PROPERTIES] + [prop for prop in EmbeddingUtils.PROPERTIES]
+        self.property_descriptors.append(self.STORE_TEXT)
+        self.property_descriptors.append(self.DISTANCE_METHOD)
+        self.property_descriptors.append(self.DOC_ID_FIELD_NAME)
+
+
+    def getPropertyDescriptors(self):
+        return self.property_descriptors
+
+    def onScheduled(self, context):
+        self.client = ChromaUtils.create_client(context)
+        self.embedding_function = EmbeddingUtils.create_embedding_function(context)
+
+
+    def transform(self, context, flowfile):
+        client = self.client
+        embedding_function = self.embedding_function
+        collection_name = context.getProperty(ChromaUtils.COLLECTION_NAME).evaluateAttributeExpressions(flowfile).getValue()
+        distance_method = context.getProperty(self.DISTANCE_METHOD).getValue()
+        id_field_name = context.getProperty(self.DOC_ID_FIELD_NAME).evaluateAttributeExpressions(flowfile).getValue()
+
+        collection = client.get_or_create_collection(
+            name=collection_name,
+            embedding_function=embedding_function,
+            metadata={"hnsw:space": distance_method})
+
+        json_lines = flowfile.getContentsAsBytes().decode()

Review Comment:
   Are there any concerns here related to memory consumption with large files? It seems like this should be streamed.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/PutChroma.py:
##########
@@ -0,0 +1,125 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import ChromaUtils
+import EmbeddingUtils
+
+
+class PutChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = """Publishes JSON data to a Chroma VectorDB. The Incoming data must be in single JSON per Line format, each with two keys: 'text' and 'metadata'.

Review Comment:
   The requirement for this specific structure with ` text` and `metadata`, with a single JSON object per line seems limiting. What do you think about introducing a Record Reader? That would provide at least one layer of flexibility. If those are the only object keys required, perhaps they don't need to be configurable, but that could be another option to consider.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",

Review Comment:
   Is this a maximum or limit? That would be worth clarifying in the description.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",
+        required=True,
+        validators=[StandardValidators.POSITIVE_INTEGER_VALIDATOR],
+        default_value="10",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    WHERE_CLAUSE = PropertyDescriptor(
+        name="Where Clause (Metadata Filter)",
+        description="A JSON representation of a Metadata Filter that can be applied against the Chroma documents in order to narrow down the documents that can be returned. " +
+                    "For example: { \"metadata_field\": \"some_value\" }",
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        required=False
+    )
+    WHERE_DOCUMENT_CLAUSE = PropertyDescriptor(
+        name="Where Document (Document Filter)",
+        description="A JSON representation of a Document Filter that can be applied against the Chroma documents' text in order to narrow down the documents that can be returned. " +
+                    "For example: { \"$contains\": \"search_string\" }",
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        required=False
+    )
+
+    client = None
+    embedding_function = None
+    include_ids = None
+    include_metadatas = None
+    include_documents = None
+    include_distances = None
+    include_embeddings = None
+    results_field = None
+
+    property_descriptors = [prop for prop in ChromaUtils.PROPERTIES] + [prop for prop in EmbeddingUtils.PROPERTIES] + [
+        QUERY,
+        NUMBER_OF_RESULTS,
+        QueryUtils.OUTPUT_STRATEGY,
+        QueryUtils.RESULTS_FIELD,
+        WHERE_CLAUSE,
+        WHERE_DOCUMENT_CLAUSE,
+        QueryUtils.INCLUDE_IDS,
+        QueryUtils.INCLUDE_METADATAS,
+        QueryUtils.INCLUDE_DOCUMENTS,
+        QueryUtils.INCLUDE_DISTANCES,
+        QueryUtils.INCLUDE_EMBEDDINGS]
+
+    def __init__(self, **kwargs):
+        pass
+
+    def getPropertyDescriptors(self):
+        return self.property_descriptors
+
+
+    def onScheduled(self, context):
+        self.client = ChromaUtils.create_client(context)
+        self.embedding_function = EmbeddingUtils.create_embedding_function(context)
+        self.include_ids = context.getProperty(QueryUtils.INCLUDE_IDS).asBoolean()
+        self.include_metadatas = context.getProperty(QueryUtils.INCLUDE_METADATAS).asBoolean()
+        self.include_documents = context.getProperty(QueryUtils.INCLUDE_DOCUMENTS).asBoolean()
+        self.include_distances = context.getProperty(QueryUtils.INCLUDE_DISTANCES).asBoolean()
+        self.include_embeddings = context.getProperty(QueryUtils.INCLUDE_EMBEDDINGS).asBoolean()
+        self.results_field = context.getProperty(QueryUtils.RESULTS_FIELD).getValue()
+        self.query_utils = QueryUtils.QueryUtils(context)
+
+    def transform(self, context, flowfile):
+        client = self.client
+        embedding_function = self.embedding_function
+        collection_name = context.getProperty(ChromaUtils.COLLECTION_NAME).evaluateAttributeExpressions(flowfile).getValue()
+
+        collection = client.get_collection(
+            name=collection_name,
+            embedding_function=embedding_function)
+
+        query_text = context.getProperty(self.QUERY).evaluateAttributeExpressions(flowfile).getValue()
+        embeddings = embedding_function([query_text])
+
+        included_fields = []
+        if self.include_distances:
+            included_fields.append('distances')
+        if self.include_documents:
+            included_fields.append('documents')
+        if self.include_embeddings:
+            included_fields.append('embeddings')
+        if self.include_metadatas:
+            included_fields.append('metadatas')
+
+        where = None
+        where_clause = context.getProperty(self.WHERE_CLAUSE).evaluateAttributeExpressions(flowfile).getValue()
+        if where_clause is not None:
+            where = json.loads(where_clause)
+
+        where_document = None
+        where_document_clause = context.getProperty(self.WHERE_DOCUMENT_CLAUSE).evaluateAttributeExpressions(flowfile).getValue()
+        if where_document_clause is not None:
+            where_document = json.loads(where_document_clause)
+
+        query_results = collection.query(
+            query_embeddings=embeddings,
+            n_results=context.getProperty(self.NUMBER_OF_RESULTS).evaluateAttributeExpressions(flowfile).asInteger(),
+            include=included_fields,
+            where_document=where_document,
+            where=where
+        )
+
+        ids = query_results['ids'][0]
+        distances = None if (not self.include_distances or query_results['distances'] is None) else query_results['distances'][0]
+        metadatas = None if (not self.include_metadatas or query_results['metadatas'] is None) else query_results['metadatas'][0]
+        documents = None if (not self.include_documents or query_results['documents'] is None) else query_results['documents'][0]
+        embeddings = None if (not self.include_embeddings or query_results['embeddings'] is None) else query_results['embeddings'][0]
+
+        (output_contents, mime_type) = self.query_utils.create_json(flowfile, documents, metadatas, embeddings, distances, ids)
+
+        # Return the results
+        attributes = {"mime.type": mime_type}

Review Comment:
   Will the response `mime.type` always be `application/json`? It would be helpful to indicate the WritesAttribute status for documentation purposes.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",
+        required=True,
+        validators=[StandardValidators.POSITIVE_INTEGER_VALIDATOR],
+        default_value="10",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    WHERE_CLAUSE = PropertyDescriptor(
+        name="Where Clause (Metadata Filter)",

Review Comment:
   Recommend avoiding the use of parentheses in the property name. What do you think about naming this simply `Metadata Filter Clause` or `Metadata Filter`? With the value needing to be JSON, `Metadata Filter` seems straightforward.



##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",
+        required=True,
+        validators=[StandardValidators.POSITIVE_INTEGER_VALIDATOR],
+        default_value="10",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    WHERE_CLAUSE = PropertyDescriptor(
+        name="Where Clause (Metadata Filter)",
+        description="A JSON representation of a Metadata Filter that can be applied against the Chroma documents in order to narrow down the documents that can be returned. " +
+                    "For example: { \"metadata_field\": \"some_value\" }",
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        required=False
+    )
+    WHERE_DOCUMENT_CLAUSE = PropertyDescriptor(
+        name="Where Document (Document Filter)",

Review Comment:
   As above, what about `Document Filter` for this property name?



##########
nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/bootstrap.conf:
##########
@@ -27,7 +27,7 @@ java.arg.3=-Xmx512m
 
 java.arg.14=-Djava.awt.headless=true
 
-#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002
+java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002

Review Comment:
   It looks like the comment needs to be restored.
   ```suggestion
   #java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #7894: NIFI-12240, NIFI-12195: Created new python processors for text embedd…
URL: https://github.com/apache/nifi/pull/7894


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377868095


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/ChunkDocuments.py:
##########
@@ -0,0 +1,211 @@
+# 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.
+
+import json
+
+from langchain.text_splitter import Language
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency, ExpressionLanguageScope
+
+SPLIT_BY_CHARACTER = 'Split by Character'
+SPLIT_CODE = 'Split Code'
+RECURSIVELY_SPLIT_BY_CHARACTER = 'Recursively Split by Character'
+
+TEXT_KEY = "text"
+METADATA_KEY = "metadata"
+
+
+class ChunkDocuments(FlowFileTransform):

Review Comment:
   Following the convention of other components, it seems like this should be named `ChunkDocument` as opposed to `ChunkDocuments`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377931964


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",
+        required=True,
+        validators=[StandardValidators.POSITIVE_INTEGER_VALIDATOR],
+        default_value="10",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    WHERE_CLAUSE = PropertyDescriptor(
+        name="Where Clause (Metadata Filter)",
+        description="A JSON representation of a Metadata Filter that can be applied against the Chroma documents in order to narrow down the documents that can be returned. " +
+                    "For example: { \"metadata_field\": \"some_value\" }",
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        required=False
+    )
+    WHERE_DOCUMENT_CLAUSE = PropertyDescriptor(
+        name="Where Document (Document Filter)",
+        description="A JSON representation of a Document Filter that can be applied against the Chroma documents' text in order to narrow down the documents that can be returned. " +
+                    "For example: { \"$contains\": \"search_string\" }",
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        required=False
+    )
+
+    client = None
+    embedding_function = None
+    include_ids = None
+    include_metadatas = None
+    include_documents = None
+    include_distances = None
+    include_embeddings = None
+    results_field = None
+
+    property_descriptors = [prop for prop in ChromaUtils.PROPERTIES] + [prop for prop in EmbeddingUtils.PROPERTIES] + [
+        QUERY,
+        NUMBER_OF_RESULTS,
+        QueryUtils.OUTPUT_STRATEGY,
+        QueryUtils.RESULTS_FIELD,
+        WHERE_CLAUSE,
+        WHERE_DOCUMENT_CLAUSE,
+        QueryUtils.INCLUDE_IDS,
+        QueryUtils.INCLUDE_METADATAS,
+        QueryUtils.INCLUDE_DOCUMENTS,
+        QueryUtils.INCLUDE_DISTANCES,
+        QueryUtils.INCLUDE_EMBEDDINGS]
+
+    def __init__(self, **kwargs):
+        pass
+
+    def getPropertyDescriptors(self):
+        return self.property_descriptors
+
+
+    def onScheduled(self, context):
+        self.client = ChromaUtils.create_client(context)
+        self.embedding_function = EmbeddingUtils.create_embedding_function(context)
+        self.include_ids = context.getProperty(QueryUtils.INCLUDE_IDS).asBoolean()
+        self.include_metadatas = context.getProperty(QueryUtils.INCLUDE_METADATAS).asBoolean()
+        self.include_documents = context.getProperty(QueryUtils.INCLUDE_DOCUMENTS).asBoolean()
+        self.include_distances = context.getProperty(QueryUtils.INCLUDE_DISTANCES).asBoolean()
+        self.include_embeddings = context.getProperty(QueryUtils.INCLUDE_EMBEDDINGS).asBoolean()
+        self.results_field = context.getProperty(QueryUtils.RESULTS_FIELD).getValue()
+        self.query_utils = QueryUtils.QueryUtils(context)
+
+    def transform(self, context, flowfile):
+        client = self.client
+        embedding_function = self.embedding_function
+        collection_name = context.getProperty(ChromaUtils.COLLECTION_NAME).evaluateAttributeExpressions(flowfile).getValue()
+
+        collection = client.get_collection(
+            name=collection_name,
+            embedding_function=embedding_function)
+
+        query_text = context.getProperty(self.QUERY).evaluateAttributeExpressions(flowfile).getValue()
+        embeddings = embedding_function([query_text])
+
+        included_fields = []
+        if self.include_distances:
+            included_fields.append('distances')
+        if self.include_documents:
+            included_fields.append('documents')
+        if self.include_embeddings:
+            included_fields.append('embeddings')
+        if self.include_metadatas:
+            included_fields.append('metadatas')
+
+        where = None
+        where_clause = context.getProperty(self.WHERE_CLAUSE).evaluateAttributeExpressions(flowfile).getValue()
+        if where_clause is not None:
+            where = json.loads(where_clause)
+
+        where_document = None
+        where_document_clause = context.getProperty(self.WHERE_DOCUMENT_CLAUSE).evaluateAttributeExpressions(flowfile).getValue()
+        if where_document_clause is not None:
+            where_document = json.loads(where_document_clause)
+
+        query_results = collection.query(
+            query_embeddings=embeddings,
+            n_results=context.getProperty(self.NUMBER_OF_RESULTS).evaluateAttributeExpressions(flowfile).asInteger(),
+            include=included_fields,
+            where_document=where_document,
+            where=where
+        )
+
+        ids = query_results['ids'][0]
+        distances = None if (not self.include_distances or query_results['distances'] is None) else query_results['distances'][0]
+        metadatas = None if (not self.include_metadatas or query_results['metadatas'] is None) else query_results['metadatas'][0]
+        documents = None if (not self.include_documents or query_results['documents'] is None) else query_results['documents'][0]
+        embeddings = None if (not self.include_embeddings or query_results['embeddings'] is None) else query_results['embeddings'][0]
+
+        (output_contents, mime_type) = self.query_utils.create_json(flowfile, documents, metadatas, embeddings, distances, ids)
+
+        # Return the results
+        attributes = {"mime.type": mime_type}

Review Comment:
   It will not always be `application/json`. The mime type will depend on the chosen Output Strategy.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377927376


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/PutChroma.py:
##########
@@ -0,0 +1,125 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope
+import ChromaUtils
+import EmbeddingUtils
+
+
+class PutChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = """Publishes JSON data to a Chroma VectorDB. The Incoming data must be in single JSON per Line format, each with two keys: 'text' and 'metadata'.

Review Comment:
   In all Gen AI examples that I've come across, the structure is JSON and has a 'text' or similar property along with a 'metadata' property. So I tried to make this as intuitive as possible. This is also the format that is produced by the ParseDocument processor. Generally, I expect the flow to look like:
   `(some source) -> ParseDocument -> ChunkDocument -> PutChroma`
   And so this format keeps things simple. Unfortunately, we do not yet have the @UseCase and @MultiProcessorUseCase capabilities built out yet for Python based processors, so this hasn't been that clearly documented yet. But this will definitely be something we'll want to highlight.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "joewitt (via GitHub)" <gi...@apache.org>.
joewitt commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1374928742


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/ParseUnstructuredDocument.py:
##########
@@ -0,0 +1,259 @@
+# 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.
+
+import io
+import json
+from typing import List
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency
+
+PLAIN_TEXT = "Plain Text"
+HTML = "HTML"
+MARKDOWN = "Markdown"
+PDF = "PDF"
+EXCEL = "Microsoft Excel"
+POWERPOINT = "Microsoft PowerPoint"
+WORD = "Microsoft Word"
+
+PARSING_STRATEGY_AUTO = "Automatic"
+PARSING_STRATEGY_HIGH_RES = "High Resolution"
+PARSING_STRATEGY_OCR_ONLY = "OCR Only"
+PARSING_STRATEGY_FAST = "Fast"
+
+SINGLE_DOCUMENT = "Single Document"
+DOCUMENT_PER_ELEMENT = "Document Per Element"
+
+TEXT_KEY = "text"
+METADATA_KEY = "metadata"
+
+
+class ParseUnstructuredDocument(FlowFileTransform):

Review Comment:
   yeah with a user hat on - i'm parsing a doc.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1374905229


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/ChromaUtils.py:
##########
@@ -0,0 +1,155 @@
+# 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.
+
+from nifiapi.properties import PropertyDescriptor, StandardValidators, PropertyDependency, ExpressionLanguageScope
+
+# Connection Strategies
+LOCAL_DISK = "Local Disk"
+REMOTE_SERVER = "Remote Chroma Server"
+
+# Authentication Strategies
+TOKEN = "Token Authentication"
+BASIC_AUTH = "Basic Authentication"
+NONE = "None"
+
+# Transport Protocols
+HTTP = "http"
+HTTPS = "https"
+
+CONNECTION_STRATEGY = PropertyDescriptor(
+    name="Connection Strategy",
+    description="Specifies how to connect to the Chroma server",
+    allowable_values=[LOCAL_DISK, REMOTE_SERVER],
+    default_value=REMOTE_SERVER,
+    required=True
+)
+DIRECTORY = PropertyDescriptor(
+    name="Directory",
+    description="The Directory that Chroma should use to persist data",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    default_value="./chroma",
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, LOCAL_DISK)]
+)
+HOSTNAME = PropertyDescriptor(
+    name="Hostname",
+    description="The hostname to connect to in order to communicate with Chroma",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    default_value="localhost",
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+PORT = PropertyDescriptor(
+    name="Port",
+    description="The port that the Chroma server is listening on",
+    validators=[StandardValidators.PORT_VALIDATOR],
+    default_value="8000",
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+TRANSPORT_PROTOCOL = PropertyDescriptor(
+    name="Transport Protocol",
+    description="Specifies whether connections should be made over http or https",
+    allowable_values=[HTTP, HTTPS],
+    default_value=HTTPS,
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+AUTH_STRATEGY = PropertyDescriptor(
+    name="Authentication Strategy",
+    description="Specifies how to authenticate to Chroma server",
+    allowable_values=[TOKEN, BASIC_AUTH, NONE],
+    default_value=TOKEN,
+    required=True,
+    dependencies=[PropertyDependency(CONNECTION_STRATEGY, REMOTE_SERVER)]
+)
+AUTH_TOKEN = PropertyDescriptor(
+    name="Authentication Token",
+    description="The token to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    sensitive=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, TOKEN)]
+)
+USERNAME = PropertyDescriptor(
+    name="Username",
+    description="The username to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, BASIC_AUTH)]
+)
+PASSWORD = PropertyDescriptor(
+    name="Password",
+    description="The password to use for authenticating to Chroma server",
+    validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+    required=True,
+    sensitive=True,
+    dependencies=[PropertyDependency(AUTH_STRATEGY, BASIC_AUTH)]
+)
+COLLECTION_NAME = PropertyDescriptor(
+    name="Collection Name",
+    description="The name of the Chroma Collection to update",

Review Comment:
   Will mention that in the description of the PutChroma Processor. It does not belong here, as this Property Descriptor is used by both PutChroma and QueryChroma. Which means it's a typo to say "the Collection to update" so will address that as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "markap14 (via GitHub)" <gi...@apache.org>.
markap14 commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1377923349


##########
nifi-python-extensions/nifi-text-embeddings-module/src/main/python/vectorstores/QueryChroma.py:
##########
@@ -0,0 +1,159 @@
+# 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.
+
+import json
+
+from nifiapi.flowfiletransform import FlowFileTransform, FlowFileTransformResult
+from nifiapi.properties import PropertyDescriptor, StandardValidators, ExpressionLanguageScope, PropertyDependency
+import ChromaUtils
+import EmbeddingUtils
+import QueryUtils
+
+
+class QueryChroma(FlowFileTransform):
+    class Java:
+        implements = ['org.apache.nifi.python.processor.FlowFileTransform']
+
+    class ProcessorDetails:
+        version = '2.0.0-SNAPSHOT'
+        description = "Queries a Chroma Vector Database in order to gather a specified number of documents that are most closely related to the given query."
+        tags = ["chroma", "vector", "vectordb", "embeddings", "enrich", "enrichment", "ai", "artificial intelligence", "ml", "machine learning", "text", "LLM"]
+
+
+
+    QUERY = PropertyDescriptor(
+        name="Query",
+        description="The query to issue to the Chroma VectorDB. The query is always converted into embeddings using the configured embedding function, and the embedding is " +
+                    "then sent to Chroma. The text itself is not sent to Chroma.",
+        required=True,
+        validators=[StandardValidators.NON_EMPTY_VALIDATOR],
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    NUMBER_OF_RESULTS = PropertyDescriptor(
+        name="Number of Results",
+        description="The number of results to return from Chroma",
+        required=True,
+        validators=[StandardValidators.POSITIVE_INTEGER_VALIDATOR],
+        default_value="10",
+        expression_language_scope=ExpressionLanguageScope.FLOWFILE_ATTRIBUTES
+    )
+    WHERE_CLAUSE = PropertyDescriptor(
+        name="Where Clause (Metadata Filter)",

Review Comment:
   Fair.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12240, NIFI-12195: Created new python processors for text embedd… [nifi]

Posted by "github-advanced-security[bot] (via GitHub)" <gi...@apache.org>.
github-advanced-security[bot] commented on code in PR #7894:
URL: https://github.com/apache/nifi/pull/7894#discussion_r1376674842


##########
nifi-nar-bundles/nifi-py4j-bundle/nifi-py4j-bridge/src/main/java/org/apache/nifi/py4j/PythonProcess.java:
##########
@@ -231,8 +236,8 @@
         final String pythonCommand = processConfig.getPythonCommand();
 
         final ProcessBuilder processBuilder = new ProcessBuilder(pythonCommand, "-m", "pip", "install", "--upgrade", "debugpy", "--target",
-            processConfig.getPythonWorkingDirectory().getAbsolutePath());
-        processBuilder.directory(virtualEnvHome.getParentFile());
+            virtualEnvHome.getAbsolutePath());
+        processBuilder.directory(virtualEnvHome);

Review Comment:
   ## Uncontrolled command line
   
   This command line depends on a [user-provided value](1).
   This command line depends on a [user-provided value](2).
   This command line depends on a [user-provided value](3).
   This command line depends on a [user-provided value](4).
   This command line depends on a [user-provided value](5).
   This command line depends on a [user-provided value](6).
   This command line depends on a [user-provided value](7).
   This command line depends on a [user-provided value](8).
   This command line depends on a [user-provided value](9).
   This command line depends on a [user-provided value](10).
   This command line depends on a [user-provided value](11).
   This command line depends on a [user-provided value](12).
   This command line depends on a [user-provided value](13).
   
   [Show more details](https://github.com/apache/nifi/security/code-scanning/56)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org