Skip to content

Commit 8bd92b5

Browse files
authored
Fix the soft-delete check and emit soft-delete log warning at most once per bucket. (#36585)
* Fix soft-delete check and emit soft-delete log warning at most once per bucket. * Append a trailing slash to the new bucket name, otherwise soft-delete policy check doesn't work. * Include bucket name in warning. * Drive-by: also log unparsed options at most once.
1 parent 38481b5 commit 8bd92b5

File tree

2 files changed

+12
-6
lines changed

2 files changed

+12
-6
lines changed

sdks/python/apache_beam/io/gcp/gcsio.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -642,7 +642,7 @@ def _updated_to_seconds(updated):
642642

643643
def is_soft_delete_enabled(self, gcs_path):
644644
try:
645-
bucket_name, _ = parse_gcs_path(gcs_path)
645+
bucket_name, _ = parse_gcs_path(gcs_path, object_optional=True)
646646
bucket = self.get_bucket(bucket_name)
647647
if (bucket.soft_delete_policy is not None and
648648
bucket.soft_delete_policy.retention_duration_seconds > 0):

sdks/python/apache_beam/options/pipeline_options.py

Lines changed: 11 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@
3838
from apache_beam.options.value_provider import StaticValueProvider
3939
from apache_beam.options.value_provider import ValueProvider
4040
from apache_beam.transforms.display import HasDisplayData
41+
from apache_beam.utils import logger
4142
from apache_beam.utils import proto_utils
4243

4344
__all__ = [
@@ -475,7 +476,7 @@ def _warn_on_unknown_options(unknown_args, parser):
475476
suggestions = difflib.get_close_matches(arg_name, all_known_options)
476477
if suggestions:
477478
msg += f". Did you mean '{suggestions[0]}'?'"
478-
_LOGGER.warning(msg)
479+
logger.log_first_n(logging.WARN, msg, key="message")
479480

480481
def get_all_options(
481482
self,
@@ -1171,7 +1172,7 @@ def _create_default_gcs_bucket(self):
11711172
return None
11721173
bucket = gcsio.get_or_create_default_gcs_bucket(self)
11731174
if bucket:
1174-
return 'gs://%s' % bucket.id
1175+
return 'gs://%s/' % bucket.id
11751176
else:
11761177
return None
11771178

@@ -1187,14 +1188,19 @@ def _warn_if_soft_delete_policy_enabled(self, arg_name):
11871188
try:
11881189
from apache_beam.io.gcp import gcsio
11891190
if gcsio.GcsIO().is_soft_delete_enabled(gcs_path):
1190-
_LOGGER.warning(
1191-
"Bucket specified in %s has soft-delete policy enabled."
1191+
logger.log_first_n(
1192+
logging.WARN,
1193+
"Bucket %s used as %s has soft-delete policy enabled."
11921194
" To avoid being billed for unnecessary storage costs, turn"
11931195
" off the soft delete feature on buckets that your Dataflow"
11941196
" jobs use for temporary and staging storage. For more"
11951197
" information, see"
11961198
" https://cloud.google.com/storage/docs/use-soft-delete"
1197-
"#remove-soft-delete-policy." % arg_name)
1199+
"#remove-soft-delete-policy.",
1200+
gcs_path,
1201+
arg_name,
1202+
n=1,
1203+
key="message")
11981204
except ImportError:
11991205
_LOGGER.warning('Unable to check soft delete policy due to import error.')
12001206

0 commit comments

Comments
 (0)