Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add Actors for Ray Dedup. #526

Merged
merged 5 commits into from
Jan 9, 2025
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
73 changes: 54 additions & 19 deletions data_juicer/ops/deduplicator/ray_basic_deduplicator.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
from pydantic import PositiveInt
import ray

from data_juicer.utils.constant import HashKeys
from data_juicer.utils.lazy_loader import LazyLoader
Expand All @@ -7,6 +7,22 @@

redis = LazyLoader('redis', 'redis')

MERSENNE_PRIME = (1 << 61) - 1


@ray.remote(scheduling_strategy='SPREAD')
class DedupSet:

def __init__(self):
self.hash_record = set()

def setnx(self, key):
if key not in self.hash_record:
pan-x-c marked this conversation as resolved.
Show resolved Hide resolved
self.hash_record.add(key)
return True
else:
return False


class RayBasicDeduplicator(Filter):
"""
Expand All @@ -19,37 +35,56 @@ class RayBasicDeduplicator(Filter):
EMPTY_HASH_VALUE = 'EMPTY'

def __init__(self,
redis_host: str = 'localhost',
redis_port: PositiveInt = 6380,
backend: str = 'ray_actor',
redis_address: str = 'redis://localhost:6379',
pan-x-c marked this conversation as resolved.
Show resolved Hide resolved
*args,
**kwargs):
"""
Initialization.
:param redis_host: the hostname of redis server
:param redis_port: the port of redis server
:param backend: the backend for dedup, either 'ray_actor' or 'redis'
:param redis_address: the address of redis server
:param args: extra args
:param kwargs: extra args
"""
super().__init__(*args, **kwargs)
self.redis_host = redis_host
self.redis_port = redis_port
# TODO: add a barrier to ensure that flushdb is performed before
# the operator is called
r = redis.StrictRedis(host=self.redis_host, port=self.redis_port, db=0)
r.flushdb(0)
self.redis_address = redis_address
self.backend = backend
if backend == 'ray_actor':
self.dedup_set_num = int(ray.cluster_resources().get('CPU') / 2)
pan-x-c marked this conversation as resolved.
Show resolved Hide resolved
self.dedup_sets = [
DedupSet.remote() for _ in range(self.dedup_set_num)
]
elif backend == 'redis':
# TODO: add a barrier to ensure that flushdb is performed before
# the operator is called
r = redis.from_url(url=self.redis_address)
r.flushdb(0)
else:
raise ValueError(f'Unknown backend: {backend}')

def calculate_hash(self, sample, context=False):
"""Calculate hash value for the sample."""
raise NotImplementedError

def compute_stats_single(self, sample, context=False):
# init redis client
r = redis.StrictRedis(host=self.redis_host, port=self.redis_port, db=0)
# compute hash
md5_value = self.calculate_hash(sample, context)
# check existing
sample[HashKeys.is_duplicate] = r.setnx(md5_value, 1)
return sample
if self.backend == 'ray_actor':
# compute hash
md5_value = self.calculate_hash(sample, context)
dedup_set_id = int.from_bytes(
md5_value.encode(),
byteorder='little') % MERSENNE_PRIME % self.dedup_set_num
# check existing
sample[HashKeys.is_unique] = \
ray.get(self.dedup_sets[dedup_set_id].setnx.remote(md5_value))
return sample
else: # redis
# init redis client
r = redis.from_url(url=self.redis_address)
# compute hash
md5_value = self.calculate_hash(sample, context)
# check existing
sample[HashKeys.is_unique] = r.setnx(md5_value, 1)
return sample

def process_single(self, sample):
return sample[HashKeys.is_duplicate]
return sample[HashKeys.is_unique]
13 changes: 6 additions & 7 deletions data_juicer/ops/deduplicator/ray_document_deduplicator.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@
import string

import regex as re
from pydantic import PositiveInt

from ..base_op import OPERATORS
from .ray_basic_deduplicator import RayBasicDeduplicator
Expand All @@ -17,24 +16,24 @@ class RayDocumentDeduplicator(RayBasicDeduplicator):
"""

def __init__(self,
redis_host: str = 'localhost',
redis_port: PositiveInt = 6380,
backend: str = 'ray_actor',
redis_address: str = 'redis://localhost:6379',
lowercase: bool = False,
ignore_non_character: bool = False,
*args,
**kwargs):
"""
Initialization method.
:param redis_host: the hostname of redis server
:param redis_port: the port of redis server
:param backend: the backend for dedup, either 'ray_actor' or 'redis'
:param redis_address: the address of redis server
:param lowercase: Whether to convert sample text to lower case
:param ignore_non_character: Whether to ignore non-alphabet
characters, including whitespaces, digits, and punctuations
:param args: extra args
:param kwargs: extra args.
"""
super().__init__(redis_host=redis_host,
redis_port=redis_port,
super().__init__(backend=backend,
redis_address=redis_address,
*args,
**kwargs)
self.lowercase = lowercase
Expand Down
13 changes: 6 additions & 7 deletions data_juicer/ops/deduplicator/ray_image_deduplicator.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
import numpy as np
from pydantic import PositiveInt

from data_juicer.utils.lazy_loader import LazyLoader
from data_juicer.utils.mm_utils import load_data_with_context, load_image
Expand Down Expand Up @@ -36,20 +35,20 @@ class RayImageDeduplicator(RayBasicDeduplicator):
"""

def __init__(self,
redis_host: str = 'localhost',
redis_port: PositiveInt = 6380,
backend: str = 'ray_actor',
redis_address: str = 'redis://localhost:6379',
method: str = 'phash',
*args,
**kwargs):
"""
Initialization.
:param redis_host: the hostname of redis server
:param redis_port: the port of redis server
:param backend: the backend for dedup, either 'ray_actor' or 'redis'
:param redis_address: the address of redis server
:param args: extra args
:param kwargs: extra args
"""
super().__init__(redis_host=redis_host,
redis_port=redis_port,
super().__init__(backend=backend,
redis_address=redis_address,
*args,
**kwargs)
if method not in HASH_METHOD:
Expand Down
14 changes: 6 additions & 8 deletions data_juicer/ops/deduplicator/ray_video_deduplicator.py
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
import hashlib

from pydantic import PositiveInt

from data_juicer.utils.mm_utils import (close_video, load_data_with_context,
load_video)

Expand All @@ -21,19 +19,19 @@ class RayVideoDeduplicator(RayBasicDeduplicator):
"""

def __init__(self,
redis_host: str = 'localhost',
redis_port: PositiveInt = 6380,
backend: str = 'ray_actor',
redis_address: str = 'redis://localhost:6379',
*args,
**kwargs):
"""
Initialization.
:param redis_host: the hostname of redis server
:param redis_port: the port of redis server
:param backend: the backend for dedup, either 'ray_actor' or 'redis'
:param redis_address: the address of redis server
:param args: extra args
:param kwargs: extra args
"""
super().__init__(redis_host=redis_host,
redis_port=redis_port,
super().__init__(backend=backend,
redis_address=redis_address,
*args,
**kwargs)

Expand Down
2 changes: 1 addition & 1 deletion data_juicer/utils/constant.py
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,7 @@ class HashKeys(object):
videohash = DEFAULT_PREFIX + 'videohash'

# duplicate flag
is_duplicate = DEFAULT_PREFIX + 'is_duplicate'
is_unique = DEFAULT_PREFIX + 'is_unique'


class InterVars(object):
Expand Down
Loading