def set(cls, key, value, execution_date, task_id, dag_id, session=None): """ Store an XCom value. :return: None """ session.expunge_all() value = VineyardXCom.serialize_value(value) # remove any duplicate XComs query = session.query(cls).filter(cls.key == key, cls.execution_date == execution_date, cls.task_id == task_id, cls.dag_id == dag_id) targets = [] for result in query.with_entities(VineyardXCom.value): targets.append(vineyard.ObjectID(BaseXCom.deserialize_value(result))) if targets: logger.info("Drop duplicates from vineyard: %s", targets) try: client = vineyard.connect(cls.options['ipc_socket']) client.delete(targets) except Exception as e: logger.error('Failed to drop duplicates from vineyard: %s', e) # step 2: remove from the underlying xcom db query.delete() session.commit() # insert new XCom session.add(VineyardXCom(key=key, value=value, execution_date=execution_date, task_id=task_id, dag_id=dag_id)) session.commit()
def post_resolve_value(result: "XCom", value: Any, session: Session = None) -> Any: ''' The :code:`post_resolve_value` runs before the return the value to the operators to prepare necessary input data for the task. The post resolution will fill-up the occurrence if remote objects by of :code:`VineyardObjectRef` with the actual (remote) value by triggering a migration. It will also record the migrated xcom value into the db as well to make sure it can be dropped properly. ''' client = vineyard.connect(VineyardXCom.options['ipc_socket']) object_id = vineyard.ObjectID(value) meta = client.get_meta(object_id) if meta.islocal: return client.get(object_id) # migration logger.debug('start migration: %r') target_id = client.migrate(object_id) logger.debug('finish migration: %r -> %r', object_id, target_id) # TODO: should we record the replicated XCom into the db ? # session.add(VineyardXCom(...)) # session.commit() return client.get(target_id)
def execute(cls, ctx, op): if vineyard is None: raise RuntimeError("vineyard is not available") socket, needs_put = resolve_vineyard_socket(ctx, op) client = vineyard.connect(socket) # some op might be fused and executed twice on different workers if not needs_put: # might be fused try: # pragma: no cover meta = ctx.get_chunks_meta([op.inputs[0].key])[0] tensor_id = vineyard.ObjectID(meta["object_ref"]) if not client.exists(tensor_id): needs_put = True except KeyError: needs_put = True if needs_put: tensor_id = client.put(ctx[op.inputs[0].key], partition_index=op.inputs[0].index) else: # pragma: no cover meta = client.get_meta(tensor_id) new_meta = vineyard.ObjectMeta() for k, v in meta.items(): if k not in ["id", "signature", "instance_id"]: if isinstance(v, vineyard.ObjectMeta): new_meta.add_member(k, v) else: new_meta[k] = v new_meta["partition_index_"] = to_json(op.inputs[0].index) tensor_id = client.create_metadata(new_meta).id client.persist(tensor_id) holder = np.empty((1, ), dtype=object) holder[0] = tensor_id ctx[op.outputs[0].key] = holder
def serialize_value(value: Any): client = vineyard.connect(VineyardXCom.options['ipc_socket']) value_id = client.put(value) if VineyardXCom.options['persist']: client.persist(value_id) logger.debug("serialize_value: %s -> %r", value, value_id) return BaseXCom.serialize_value(repr(value_id))
def tile(cls, op): check_chunks_unknown_shape(op.inputs, TilesError) if vineyard is None: raise RuntimeError('vineyard is not available') client = vineyard.connect(op.vineyard_socket) ctx = get_context() if ctx.running_mode == RunningMode.distributed: metas = ctx.get_worker_metas() workers = {meta['vineyard']['instance_id']: addr for addr, meta in metas.items()} else: workers = {client.instance_id: '127.0.0.1'} all_chunk_ids = ctx.get_chunk_results([c.key for c in op.inputs[0].chunks]) all_chunk_ids = pd.concat(all_chunk_ids, axis='index') chunk_map = defaultdict(list) for instance_id, chunk_id in all_chunk_ids.itertuples(index=False): chunk_map[instance_id].append(chunk_id) out_chunks = [] for idx, (instance_id, local_chunks) in enumerate(chunk_map.items()): chunk_op = op.copy().reset_key() chunk_op._local_chunks = local_chunks chunk_op._expect_worker = workers[instance_id] out_chunks.append(chunk_op.new_chunk(op.inputs[0].chunks, shape=(1,), index=(idx,))) out_chunks = cls._process_out_chunks(op, out_chunks) new_op = op.copy() return new_op.new_dataframes(op.inputs, shape=(len(out_chunks),), dtypes=op.inputs[0].dtypes, index_value=op.inputs[0].index_value, columns_value=op.inputs[0].columns_value, chunks=out_chunks, nsplits=((1,) * len(out_chunks),))
def execute(cls, ctx, op): if vineyard is None: raise RuntimeError('vineyard is not available') socket, needs_put = resolve_vineyard_socket(ctx, op) client = vineyard.connect(socket) # some op might be fused and executed twice on different workers if not needs_put: # might be fused try: # pragma: no cover meta = ctx.get_chunks_meta([op.inputs[0].key])[0] df_id = vineyard.ObjectID(meta['object_ref']) if not client.exists(df_id): needs_put = True except KeyError: needs_put = True if needs_put: df_id = client.put(ctx[op.inputs[0].key], partition_index=op.inputs[0].index) else: # pragma: no cover meta = client.get_meta(df_id) new_meta = vineyard.ObjectMeta() for k, v in meta.items(): if k not in ['id', 'signature', 'instance_id']: if isinstance(v, vineyard.ObjectMeta): new_meta.add_member(k, v) else: new_meta[k] = v new_meta['partition_index_'] = to_json(op.inputs[0].index) df_id = client.create_metadata(new_meta).id client.persist(df_id) ctx[op.outputs[0].key] = pd.DataFrame({0: [df_id]})
def read_bytes_collection( vineyard_socket, prefix, storage_options, proc_num, proc_index ): """Read a set of files as a collection of ByteStreams.""" client = vineyard.connect(vineyard_socket) protocol, prefix_path = split_protocol(prefix) fs = fsspec.filesystem(protocol, **storage_options) worker_prefix = os.path.join(prefix_path, '%s-%s' % (proc_num, proc_index)) logger.info("start creating blobs ...") queue: "ConcurrentQueue[Tuple[ByteStream, str]]" = ConcurrentQueue() stream_id = read_stream_collections(client, fs, queue, worker_prefix, worker_prefix) client.persist(stream_id) report_success(stream_id) logger.info("start reading blobs ...") executor = ThreadStreamExecutor( ReadToByteStreamExecutor, parallism=1, client=client, fs=fs, task_queue=queue, chunk_size=CHUNK_SIZE, ) executor.execute()
def read_vineyard_dataframe(vineyard_socket, path, storage_options, read_options, proc_num, proc_index): client = vineyard.connect(vineyard_socket) params = dict() if storage_options: raise ValueError("Read vineyard current not support storage options") params["header_row"] = "1" if read_options.get("header_row", False) else "0" params["delimiter"] = bytes(read_options.get("delimiter", ","), "utf-8").decode("unicode_escape") stream = DataframeStream.new(client, params) client.persist(stream.id) report_success(stream.id) name = urlparse(path).netloc # the "name" part in URL can be a name, or an ObjectID for convenience. try: df_id = client.get_name(name) except Exception: df_id = vineyard.ObjectID(name) dataframes = client.get(df_id) writer: DataframeStream.Writer = stream.open_writer(client) try: for df in dataframes: batch = pa.RecordBatch.from_pandas(df) writer.write(batch) writer.finish() except Exception: report_exception() writer.fail() sys.exit(-1)
def write_hdfs_orc(vineyard_socket, stream_id, path, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num or streams[proc_index] is None: raise ValueError( f'Fetch stream error with proc_num={proc_num},proc_index={proc_index}' ) instream = streams[proc_index] reader = instream.open_reader(client) host, port = urlparse(path).netloc.split(':') hdfs = HDFileSystem(host=host, port=int(port)) path = urlparse(path).path writer = None with hdfs.open(path, 'wb') as f: while True: try: buf = reader.next() except: writer.close() break buf_reader = pa.ipc.open_stream(buf) if writer is None: #get schema schema = {} for field in buf_reader.schema: schema[field.name] = orc_type(field.type) writer = pyorc.Writer(f, pyorc.Struct(**schema)) for batch in buf_reader: df = batch.to_pandas() writer.writerows(df.itertuples(False, None))
def execute(cls, ctx, op): if vineyard is None: raise RuntimeError('vineyard is not available') client = vineyard.connect(op.vineyard_socket) meta = vineyard.ObjectMeta() instances = set() chunks = set() for idx, in_chunk in enumerate(op.inputs): instance_id, chunk_id = ctx[in_chunk.key] instances.add(instance_id) chunks.add(chunk_id) meta.add_member('object_%d' % idx, vineyard.ObjectID(chunk_id)) meta['typename'] = 'vineyard::ObjectSet' meta['num_of_instances'] = len(instances) meta['num_of_objects'] = len(chunks) object_set_id = client.create_metadata(meta) meta = vineyard.ObjectMeta() meta['typename'] = 'vineyard::GlobalTensor<%s>' % op.dtype.name meta['shape_'] = json.dumps(op.shape) meta['chunk_shape_'] = json.dumps(op.chunk_shape) meta.add_member('chunks_', object_set_id) global_tensor_id = client.create_metadata(meta) client.persist(global_tensor_id) # # store the result object id to execution context ctx[op.outputs[0].key] = repr(global_tensor_id)
def test_seriarialize_round_trip_on_oss(vineyard_ipc_socket, vineyard_endpoint, global_obj): accessKeyID = os.environ["ACCESS_KEY_ID"] accessKeySecret = os.environ["SECRET_ACCESS_KEY"] endpoint = os.environ.get("ENDPOINT", "http://oss-cn-hangzhou.aliyuncs.com") vineyard.io.serialize('oss://grape-uk/tmp/seri-test', global_obj, vineyard_ipc_socket=vineyard_ipc_socket, vineyard_endpoint=vineyard_endpoint, storage_options={ "key": accessKeyID, "secret": accessKeySecret, "endpoint": endpoint, }) ret = vineyard.io.deserialize('oss://grape-uk/tmp/seri-test', vineyard_ipc_socket=vineyard_ipc_socket, vineyard_endpoint=vineyard_endpoint, storage_options={ "key": accessKeyID, "secret": accessKeySecret, "endpoint": endpoint, }) client = vineyard.connect(vineyard_ipc_socket) old_meta = client.get_meta(global_obj) new_meta = client.get_meta(ret) print('old meta', old_meta) print('new meta', new_meta)
def write_bytes_collection(vineyard_socket, prefix, stream_id, storage_options, proc_num, proc_index): """Read bytes from stream and write to external storage. Raises: ValueError: If the stream is invalid. """ client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num: report_error("Expected: %s stream partitions" % proc_num) sys.exit(-1) worker_prefix = os.path.join(prefix, '%s-%s' % (proc_num, proc_index)) # collect all blobs, and prepare metadata queue: "ConcurrentQueue[ObjectID]" = ConcurrentQueue() write_stream_collections(client, streams[proc_index].id, queue, worker_prefix, storage_options) # write streams to file executor = ThreadStreamExecutor( WriteBytesExecutor, parallism=1, client=client, prefix=worker_prefix, storage_options=storage_options, task_queue=queue, ) executor.execute()
def serialize(vineyard_socket, object_id): '''Serialize a vineyard object as a stream. The serialization executes in the following steps: 1. glob all blobs in the meta 2. build a stream for each blob 3. generate a hierarchical `StreamCollection` object as the result ''' client = vineyard.connect(vineyard_socket) meta = client.get_meta(object_id) queue: "ConcurrentQueue[Tuple[ByteStream, memoryview]]" = ConcurrentQueue() serialized_id = traverse_to_serialize(client, meta, queue, '') # object id done client.persist(serialized_id) report_success(serialized_id) # start transfer # # easy to be implemented as a threaded executor in a future executor = ThreadStreamExecutor(SerializeExecutor, parallism=1, task_queue=queue) results = executor.execute() logger.info('finish serialization: %s', results)
def parse_dataframe(vineyard_socket, stream_id, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num or streams[proc_index] is None: raise ValueError(f'Fetch stream error with proc_num={proc_num},proc_index={proc_index}') instream = streams[proc_index] stream_reader = instream.open_reader(client) header_row = (instream.params.get('header_row', None) == '1') delimiter = instream.params.get('delimiter', ',') builder = ByteStreamBuilder(client) stream = builder.seal(client) ret = {'type': 'return'} ret['content'] = repr(stream.id) print(json.dumps(ret)) stream_writer = stream.open_writer(client) first_write = header_row while True: try: content = stream_reader.next() except: stream_writer.finish() break buf_reader = pa.ipc.open_stream(content) for batch in buf_reader: df = batch.to_pandas() buf = df.to_csv(header=first_write, index=False, sep=delimiter).encode() first_write = False chunk = stream_writer.next(len(buf)) buf_writer = pa.FixedSizeBufferWriter(chunk) buf_writer.write(buf) buf_writer.close()
def deserialize(vineyard_socket, object_id, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(object_id) if len(streams) != proc_num: report_error("Expected: %s stream partitions" % proc_num) sys.exit(-1) queue: "ConcurrentQueue[Tuple[ByteStream, Union[BlobBuilder, Blob]]]" = ( ConcurrentQueue()) traverse_to_prepare(client, streams[proc_index].id, queue) # serve as a stream id -> blob id mapping rqueue: "ConcurrentQueue[Tuple[ObjectID, str, Blob]]" = ConcurrentQueue() # copy blobs executor = ThreadStreamExecutor( ReconstructExecututor, parallism=1, client=client, task_queue=queue, result_queue=rqueue, ) executor.execute() blobs: Dict[ObjectID, Blob] = dict() while not rqueue.empty(): bs, memberpath, blob = rqueue.get(block=False) blobs[bs] = (memberpath, blob) _, result = traverse_to_rebuild(client, streams[proc_index].id, blobs) client.persist(result.id) report_success(result.id)
def write_vineyard_dataframe(vineyard_socket, stream_id, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num or streams[proc_index] is None: raise ValueError( f"Fetch stream error with proc_num={proc_num},proc_index={proc_index}" ) instream = streams[proc_index] stream_reader = instream.open_reader(client) idx = 0 while True: try: content = stream_reader.next() except: break buf_reader = pa.ipc.open_stream(content) while True: try: batch = buf_reader.read_next_batch() except StopIteration: break df = batch.to_pandas() df_id = client.put(df, partition_index=[proc_index, 0], row_batch_index=idx) client.persist(df_id) idx += 1 print_vineyard_id(df_id)
def execute(cls, ctx, op): if vineyard is None: raise RuntimeError('vineyard is not available') client = vineyard.connect(op.vineyard_socket) # setup builder context from vineyard.core import default_builder_context, default_resolver_context from vineyard.data.dataframe import register_dataframe_types from vineyard.data.tensor import register_tensor_types register_dataframe_types(builder_ctx=default_builder_context, resolver_ctx=default_resolver_context) register_tensor_types(builder_ctx=default_builder_context, resolver_ctx=default_resolver_context) if options.vineyard.enabled and op.vineyard_object_id: # the chunk already exists in vineyard df_id = vineyard.ObjectID(op.vineyard_object_id) else: df_id = client.put(ctx[op.inputs[0].key], partition_index=op.inputs[0].index) client.persist(df_id) # store the result object id to execution context ctx[op.outputs[0].key] = (client.instance_id, repr(df_id))
def vineyard_client(request): ipc_socket = request.config.option.vineyard_ipc_socket rpc_endpoint = request.config.option.vineyard_rpc_endpoint if rpc_endpoint is not None: return request.connect(rpc_endpoint) else: return vineyard.connect(ipc_socket)
def test_seriarialize_round_trip_on_s3(vineyard_ipc_socket, vineyard_endpoint, global_obj): accessKeyID = os.environ["ACCESS_KEY_ID"] accessKeySecret = os.environ["SECRET_ACCESS_KEY"] region_name = os.environ.get("REGION", "us-east-1") stream = vineyard.io.serialize( "s3://test-bucket/tmp/seri-test", global_obj, vineyard_ipc_socket=vineyard_ipc_socket, vineyard_endpoint=vineyard_endpoint, storage_options={ "key": accessKeyID, "secret": accessKeySecret, "client_kwargs": { "region_name": region_name }, }, ) ret = vineyard.io.deserialize( 's3://test-bucket/tmp/seri-test', vineyard_ipc_socket=vineyard_ipc_socket, vineyard_endpoint=vineyard_endpoint, storage_options={ "key": accessKeyID, "secret": accessKeySecret, "client_kwargs": { "region_name": region_name }, }, ) client = vineyard.connect(vineyard_ipc_socket) old_meta = client.get_meta(global_obj) new_meta = client.get_meta(ret) print('old meta', old_meta) print('new meta', new_meta)
def write_orc(vineyard_socket, path, stream_id, storage_options, write_options, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num or streams[proc_index] is None: raise ValueError( f"Fetch stream error with proc_num={proc_num},proc_index={proc_index}" ) instream = streams[proc_index] reader = instream.open_reader(client) writer = None path += f"_{proc_index}" with fsspec.open(path, "wb", **storage_options) as f: while True: try: buf = reader.next() except vineyard.StreamDrainedException: writer.close() break buf_reader = pa.ipc.open_stream(buf) if writer is None: # get schema schema = {} for field in buf_reader.schema: schema[field.name] = orc_type(field.type) writer = pyorc.Writer(f, pyorc.Struct(**schema)) while True: try: batch = buf_reader.read_next_batch() except StopIteration: break df = batch.to_pandas() writer.writerows(df.itertuples(False, None))
def write_oss_bytes(vineyard_socket, stream_id, path, proc_num, proc_index): client = vineyard.connect(vineyard_socket) streams = client.get(stream_id) if len(streams) != proc_num or streams[proc_index] is None: raise ValueError( f'Fetch stream error with proc_num={proc_num},proc_index={proc_index}' ) instream = streams[proc_index] reader = instream.open_reader(client) parsed = urlparse(path) auth = oss2.Auth(parsed.username, parsed.password) _, bucket_name, object_name = parsed.path.split('/', 2) bucket = oss2.Bucket(auth, parsed.hostname, bucket_name) result = None object_name += f'_{proc_index}' while True: try: buf = reader.next() except vineyard.StreamDrainedException: break if result is None: offset = 0 else: offset = result.next_position result = bucket.append_object(object_name, offset, bytes(memoryview(buf)))
def test_migrate_stream(vineyard_ipc_sockets, vineyard_endpoint, test_dataset, test_dataset_tmp): vineyard_ipc_sockets = list( itertools.islice(itertools.cycle(vineyard_ipc_sockets), 2)) # read the file as a stream, note that the open api # always returns a global stream stream = vineyard.io.open( "file://%s/p2p-31.e" % test_dataset, vineyard_ipc_socket=vineyard_ipc_sockets[0], vineyard_endpoint=vineyard_endpoint, read_options={ "header_row": False, "delimiter": " " }, ) # extract the local stream from the opened global stream client1 = vineyard.connect(vineyard_ipc_sockets[0]) local_streams = client1.get(stream) # migrate the local stream to another vineyardd client2 = vineyard.connect(vineyard_ipc_sockets[1]) new_stream = client2.migrate_stream(local_streams[0].id) # create a global stream from the migrated local stream to fit # the open api meta = vineyard.ObjectMeta() meta['typename'] = 'vineyard::ParallelStream' meta.set_global(True) meta['size_'] = 1 meta.add_member("stream_0", new_stream) ret_id = client2.create_metadata(meta) client2.persist(ret_id) # output the global stream vineyard.io.open( "file://%s/p2p-31.out" % test_dataset_tmp, ret_id, mode="w", vineyard_ipc_socket=vineyard_ipc_sockets[1], vineyard_endpoint=vineyard_endpoint, ) # check the equility assert filecmp.cmp("%s/p2p-31.e" % test_dataset, "%s/p2p-31.out_0" % test_dataset_tmp)
def read_bytes( vineyard_socket: str, path: str, storage_options: Dict, read_options: Dict, proc_num: int, proc_index: int, ): client = vineyard.connect(vineyard_socket) builder = ByteStreamBuilder(client) header_row = read_options.get("header_row", False) for k, v in read_options.items(): if k in ("header_row", "include_all_columns"): builder[k] = "1" if v else "0" elif k == "delimiter": builder[k] = bytes(v, "utf-8").decode("unicode_escape") else: builder[k] = v offset = 0 chunk_size = 1024 * 1024 * 4 of = fsspec.open(path, mode="rb", **storage_options) with of as f: header_line = read_block(f, 0, 1, b'\n') builder["header_line"] = header_line.decode("unicode_escape") if header_row: offset = len(header_line) stream = builder.seal(client) client.persist(stream) ret = {"type": "return", "content": repr(stream.id)} print(json.dumps(ret), flush=True) writer = stream.open_writer(client) try: total_size = f.size() except TypeError: total_size = f.size part_size = (total_size - offset) // proc_num begin = part_size * proc_index + offset end = min(begin + part_size, total_size) if proc_index == 0: begin -= int(header_row) while begin < end: buf = read_block(f, begin, min(chunk_size, end - begin), delimiter=b"\n") size = len(buf) if not size: break begin += size - 1 chunk = writer.next(size) buf_writer = pa.FixedSizeBufferWriter(chunk) buf_writer.write(buf) buf_writer.close() writer.finish()
def __init__(self, storage_ctx, proc_id=None): StorageHandler.__init__(self, storage_ctx, proc_id=proc_id) self._client = vineyard.connect(options.vineyard.socket) logger.debug('find mapper ref: %s', VineyardKeyMapActor.default_uid()) self._mapper_ref = self._storage_ctx.actor_ctx.actor_ref( VineyardKeyMapActor.default_uid()) logger.debug('find mapper ref done: %s', VineyardKeyMapActor.default_uid())
def execute(cls, ctx, op): if vineyard is None: raise RuntimeError('vineyard is not available') client = vineyard.connect(op.vineyard_socket) # object ids vector as np.ndarray ctx[op.outputs[0].key] = pd.DataFrame( np.array(client.get(op.object_id), copy=False))
def put_partition(v, partition_info=None): client = vineyard.connect() obj_id = client.put(v, partition_index=(partition_info['number'], 0)) client.persist(obj_id) return pd.DataFrame([{ 'no': partition_info['number'], 'id': int(obj_id) }])
def test_local_single(): client = vineyard.init() obj_id = client.put(1024) client1 = vineyard.connect() assert client1.get(obj_id) == 1024 client2 = vineyard.get_current_client() assert client == client2 vineyard.shutdown()
def read_hdfs_bytes(vineyard_socket, path, proc_num, proc_index): if proc_index: return client = vineyard.connect(vineyard_socket) builder = ByteStreamBuilder(client) host, port = urlparse(path).netloc.split(':') hdfs = HDFileSystem(host=host, port=int(port), pars={"dfs.client.read.shortcircuit": "false"}) header_row = False fragments = urlparse(path).fragment.split('&') path = urlparse(path).path for frag in fragments: try: k, v = frag.split('=') except: pass else: if k == 'header_row': header_row = (v.upper() == 'TRUE') if header_row: builder[k] = '1' else: builder[k] = '0' elif k == 'delimiter': builder[k] = bytes(v, "utf-8").decode("unicode_escape") offset = 0 length = 1024 * 1024 if header_row: header_line = hdfs.read_block(path, 0, 1, b'\n') builder['header_line'] = header_line.decode('unicode_escape') offset = len(header_line)-1 stream = builder.seal(client) ret = {'type': 'return'} ret['content'] = repr(stream.id) print(json.dumps(ret)) writer = stream.open_writer(client) while True: buf = hdfs.read_block(path, offset, length, b'\n') size = len(buf) if not size: break offset += size chunk = writer.next(size) buf_writer = pa.FixedSizeBufferWriter(chunk) buf_writer.write(buf) buf_writer.close() writer.finish()
def start_requests(rs, state, ipc_socket): jobs = [job1, job2, job3, job4, job5] client = vineyard.connect(ipc_socket).fork() for _ in range(job_per_proc): if state.value != 0: break job = random.choice(jobs) job(rs, state, client)
def read_orc( vineyard_socket, path, storage_options: Dict, read_options: Dict, proc_num, proc_index, ): # This method is to read the data files of a specific hive table # that is stored as orc format in HDFS. # # In general, the data files of a hive table are stored at the hive # space in the HDFS with the table name as the directory, # e.g., # # .. code:: python # # '/user/hive/warehouse/sometable' # # To read the entire table, simply use 'hive://user/hive/warehouse/sometable' # as the path. # # In case the table is partitioned, use the sub-directory of a specific partition # to read only the data from that partition. For example, sometable is partitioned # by column date, we can read the data in a given date by giving path as # # .. code:: python # # 'hive://user/hive/warehouse/sometable/date=20201112' # if proc_index: raise ValueError("Parallel reading ORC hasn't been supported yet") if read_options: raise ValueError("Reading ORC doesn't support read options.") client = vineyard.connect(vineyard_socket) builder = DataframeStreamBuilder(client) stream = builder.seal(client) client.persist(stream) ret = {"type": "return", "content": repr(stream.id)} print(json.dumps(ret), flush=True) writer = stream.open_writer(client) parsed = urlparse(path) fs = fsspec.filesystem(parsed.scheme, **storage_options) if fs.isfile(parsed.path): files = [parsed.path] else: files = [f for f in fs.ls(parsed.path, detail=False) if fs.isfile(f)] for file_path in files: read_single_orc(file_path, fs, writer) # hdfs = HDFileSystem( # host=host, port=int(port), pars={"dfs.client.read.shortcircuit": "false"} # ) writer.finish()