Skip to content

Commit

Permalink
Print runner IP.
Browse files Browse the repository at this point in the history
  • Loading branch information
claudevdm committed Feb 10, 2025
1 parent ca6becb commit 2dbb9d4
Show file tree
Hide file tree
Showing 4 changed files with 14 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,15 +31,12 @@
from apache_beam.testing.util import equal_to

# pylint: disable=ungrouped-imports
try:
from google.api_core.exceptions import BadRequest
from apache_beam.transforms.enrichment import Enrichment
from apache_beam.ml.rag.enrichment.bigquery_vector_search import \
BigQueryVectorSearchEnrichmentHandler
from apache_beam.ml.rag.enrichment.bigquery_vector_search import \
BigQueryVectorSearchParameters
except ImportError:
raise unittest.SkipTest('BigQuery dependencies not installed')
from google.api_core.exceptions import BadRequest
from apache_beam.transforms.enrichment import Enrichment
from apache_beam.ml.rag.enrichment.bigquery_vector_search import \
BigQueryVectorSearchEnrichmentHandler
from apache_beam.ml.rag.enrichment.bigquery_vector_search import \
BigQueryVectorSearchParameters

_LOGGER = logging.getLogger(__name__)

Expand Down
9 changes: 5 additions & 4 deletions sdks/python/apache_beam/ml/rag/ingestion/alloydb_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@
])
registry.register_coder(MetadataConflictRow, RowCoder)

_LOGGER = logging.getLogger(__name__)
VECTOR_SIZE = 768


Expand Down Expand Up @@ -146,12 +147,12 @@ class AlloyDBVectorWriterConfigTest(unittest.TestCase):

@classmethod
def setUpClass(cls):
cls.host = os.environ.get('ALLOYDB_HOST', '10.0.0.2')
cls.host = os.environ.get('ALLOYDB_HOST', '10.119.0.22')
cls.port = os.environ.get('ALLOYDB_PORT', '5432')
cls.database = os.environ.get('ALLOYDB_DATABASE', 'postgres')
cls.username = os.environ.get('ALLOYDB_USERNAME', 'postgres')
if not os.environ.get('ALLOYDB_PASSWORD'):
raise ValueError("ALLOYDB_PASSWORD is not set.")
raise ValueError('ALLOYDB_PASSWORD env not set')
cls.password = os.environ.get('ALLOYDB_PASSWORD')

# Create unique table name suffix
Expand Down Expand Up @@ -373,7 +374,7 @@ def custom_row_to_chunk(row):
# Extract timestamp from custom_id
timestamp = row.custom_id.split('timestamp_')[1]
# Extract index from timestamp
i = int(timestamp.split('T wrong')[1][:2])
i = int(timestamp.split('T')[1][:2])

# Parse embedding vector
embedding_list = [
Expand Down Expand Up @@ -488,7 +489,7 @@ def custom_row_to_chunk(row):

# Verify count
count_result = rows | "Count All" >> beam.combiners.Count.Globally()
assert_that(count_result, equal_to([num_records]), label='count_check')
assert_that(count_result, equal_to([10]), label='count_check')

chunks = rows | "To Chunks" >> beam.Map(custom_row_to_chunk)
assert_that(chunks, equal_to(test_chunks), label='chunks_check')
Expand Down
4 changes: 2 additions & 2 deletions sdks/python/scripts/run_pytest.sh
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,10 @@ echo "pytest_args: $pytest_args"
echo "posargs: $posargs"

# Run with pytest-xdist and without.
pytest -o junit_suite_name=${envname} \
pytest -v -rs -o junit_suite_name=${envname} \
--junitxml=pytest_${envname}.xml -m 'not no_xdist' -n 6 --import-mode=importlib ${pytest_args} --pyargs ${posargs}
status1=$?
pytest -o junit_suite_name=${envname}_no_xdist \
pytest -v -rs -o junit_suite_name=${envname}_no_xdist \
--junitxml=pytest_${envname}_no_xdist.xml -m 'no_xdist' --import-mode=importlib ${pytest_args} --pyargs ${posargs}
status2=$?

Expand Down
1 change: 1 addition & 0 deletions sdks/python/tox.ini
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,7 @@ commands =
# many packages do not support py3.12
# Don't set TMPDIR to avoid "AF_UNIX path too long" errors in certain tests.
setenv =
passenv = ALLOYDB_PASSWORD
extras = test,gcp,dataframe,p312_ml_test
commands =
# Log tensorflow version for debugging
Expand Down

0 comments on commit 2dbb9d4

Please sign in to comment.