[Core][Optimization] change copy-on-write from dict[int, list] to list (#4648)

This commit is contained in:
youkaichao 2024-05-07 11:06:32 -07:00 committed by GitHub
parent 10760da800
commit 469f85c782
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 44 additions and 44 deletions

View File

@ -410,8 +410,7 @@ def test_cow(block_size: int, sequence_len: int, append_len: int,
expected_src = static_block_table.physical_block_ids[cow_block_id] expected_src = static_block_table.physical_block_ids[cow_block_id]
expected_dst = appender_block_table.physical_block_ids[cow_block_id] expected_dst = appender_block_table.physical_block_ids[cow_block_id]
assert expected_src in cows assert (expected_src, expected_dst) in cows
assert expected_dst in cows[expected_src]
else: else:
# Otherwise, there should be no copy-on-write. # Otherwise, there should be no copy-on-write.
assert not cows assert not cows
@ -490,8 +489,7 @@ def test_cow_lookahead_simple(block_size: int, sequence_len: int,
expected_src = static_block_table.physical_block_ids[cow_block_id] expected_src = static_block_table.physical_block_ids[cow_block_id]
expected_dst = appender_block_table.physical_block_ids[cow_block_id] expected_dst = appender_block_table.physical_block_ids[cow_block_id]
assert expected_src in cows assert (expected_src, expected_dst) in cows
assert expected_dst in cows[expected_src]
static_block_table.free() static_block_table.free()
appender_block_table.free() appender_block_table.free()

View File

@ -1,4 +1,5 @@
import time import time
from collections import defaultdict
from typing import List from typing import List
import pytest import pytest
@ -155,7 +156,10 @@ def test_append_slot_cow():
cows = block_manager.append_slots(child) cows = block_manager.append_slots(child)
assert cows assert cows
for src_block, dst_blocks in cows.items(): dict_cows = defaultdict(list)
for src_block, dst_block in cows:
dict_cows[src_block].append(dst_block)
for src_block, dst_blocks in dict_cows.items():
assert src_block not in dst_blocks assert src_block not in dst_blocks
after_blocks = block_manager.get_num_free_gpu_blocks() after_blocks = block_manager.get_num_free_gpu_blocks()

View File

@ -636,7 +636,7 @@ def test_schedule_decode_blocks_to_copy_update():
# The last request should be swapped out. # The last request should be swapped out.
scheduler.block_manager.append_slots = MagicMock() scheduler.block_manager.append_slots = MagicMock()
scheduler.block_manager.append_slots.return_value = {2: [3]} scheduler.block_manager.append_slots.return_value = [(2, 3)]
budget = create_token_budget() budget = create_token_budget()
remaining_running, output = scheduler._schedule_running( remaining_running, output = scheduler._schedule_running(
@ -845,7 +845,7 @@ def test_schedule_swapped_blocks_to_copy():
# The last request should be swapped out. # The last request should be swapped out.
scheduler.block_manager.append_slots = MagicMock() scheduler.block_manager.append_slots = MagicMock()
scheduler.block_manager.append_slots.return_value = {2: [3]} scheduler.block_manager.append_slots.return_value = [(2, 3)]
budget = create_token_budget() budget = create_token_budget()
remaining_swapped, output = scheduler._schedule_swapped( remaining_swapped, output = scheduler._schedule_swapped(

View File

@ -1,5 +1,4 @@
from collections import defaultdict from typing import Dict, Iterable, List, Optional, Protocol, Tuple
from typing import Dict, Iterable, List, Optional, Protocol
from vllm.core.block.interfaces import Block, BlockAllocator from vllm.core.block.interfaces import Block, BlockAllocator
@ -111,7 +110,7 @@ class CopyOnWriteTracker:
refcounter: RefCounterProtocol, refcounter: RefCounterProtocol,
allocator: BlockAllocator, allocator: BlockAllocator,
): ):
self._copy_on_writes: Dict[BlockId, List[BlockId]] = defaultdict(list) self._copy_on_writes: List[Tuple[BlockId, BlockId]] = []
self._refcounter = refcounter self._refcounter = refcounter
self._allocator = allocator self._allocator = allocator
@ -152,25 +151,25 @@ class CopyOnWriteTracker:
# Track src/dst copy. # Track src/dst copy.
assert src_block_id is not None assert src_block_id is not None
assert block_id is not None assert block_id is not None
self._copy_on_writes[src_block_id].append(block_id) self._copy_on_writes.append((src_block_id, block_id))
return block_id return block_id
def clear_cows(self) -> Dict[BlockId, List[BlockId]]: def clear_cows(self) -> List[Tuple[BlockId, BlockId]]:
"""Clears the copy-on-write tracking information and returns the current """Clears the copy-on-write tracking information and returns the current
state. state.
This method returns a dictionary mapping source block indices to lists This method returns a list mapping source block indices to
of destination block indices for the current copy-on-write operations. destination block indices for the current copy-on-write operations.
It then clears the internal tracking information. It then clears the internal tracking information.
Returns: Returns:
Dict[BlockId, List[BlockId]]: A dictionary mapping source List[Tuple[BlockId, BlockId]]: A list mapping source
block indices to lists of destination block indices for the block indices to destination block indices for the
current copy-on-write operations. current copy-on-write operations.
""" """
cows = dict(self._copy_on_writes) cows = self._copy_on_writes
self._copy_on_writes.clear() self._copy_on_writes = []
return cows return cows

View File

@ -1,4 +1,4 @@
from typing import Dict, FrozenSet, List, Optional from typing import Dict, FrozenSet, List, Optional, Tuple
from vllm.core.block.interfaces import (Block, BlockAllocator, BlockId, from vllm.core.block.interfaces import (Block, BlockAllocator, BlockId,
DeviceAwareBlockAllocator) DeviceAwareBlockAllocator)
@ -185,13 +185,13 @@ class CpuGpuBlockAllocator(DeviceAwareBlockAllocator):
def get_num_total_blocks(self, device: Device) -> int: def get_num_total_blocks(self, device: Device) -> int:
return self._allocators[device].get_num_total_blocks() return self._allocators[device].get_num_total_blocks()
def clear_copy_on_writes(self) -> Dict[int, List[int]]: def clear_copy_on_writes(self) -> List[Tuple[int, int]]:
"""Clears the copy-on-write (CoW) state and returns the mapping of """Clears the copy-on-write (CoW) state and returns the mapping of
source to destination block IDs. source to destination block IDs.
Returns: Returns:
Dict[int, List[int]]: A dictionary mapping source block IDs to lists List[Tuple[int, int]]: A list mapping source block IDs to
of destination block IDs. destination block IDs.
""" """
# CoW only supported on GPU # CoW only supported on GPU
device = Device.GPU device = Device.GPU

View File

@ -1,5 +1,5 @@
from abc import ABC, abstractmethod from abc import ABC, abstractmethod
from typing import Dict, FrozenSet, List, Optional, Protocol from typing import FrozenSet, List, Optional, Protocol, Tuple
from vllm.utils import Device from vllm.utils import Device
@ -122,7 +122,7 @@ class BlockAllocator(ABC):
pass pass
@abstractmethod @abstractmethod
def clear_copy_on_writes(self) -> Dict[int, List[int]]: def clear_copy_on_writes(self) -> List[Tuple[int, int]]:
pass pass
@abstractmethod @abstractmethod
@ -187,7 +187,7 @@ class DeviceAwareBlockAllocator(ABC):
pass pass
@abstractmethod @abstractmethod
def clear_copy_on_writes(self) -> Dict[int, List[int]]: def clear_copy_on_writes(self) -> List[Tuple[int, int]]:
pass pass
@abstractmethod @abstractmethod

View File

@ -1,4 +1,4 @@
from typing import Dict, FrozenSet, Iterable, List, Optional, Set from typing import FrozenSet, Iterable, List, Optional, Set, Tuple
from vllm.core.block.common import (CopyOnWriteTracker, RefCounter, from vllm.core.block.common import (CopyOnWriteTracker, RefCounter,
get_all_blocks_recursively) get_all_blocks_recursively)
@ -175,12 +175,12 @@ class NaiveBlockAllocator(BlockAllocator):
""" """
return self._cow_tracker.cow_block_if_not_appendable(block) return self._cow_tracker.cow_block_if_not_appendable(block)
def clear_copy_on_writes(self) -> Dict[BlockId, List[BlockId]]: def clear_copy_on_writes(self) -> List[Tuple[BlockId, BlockId]]:
"""Returns the copy-on-write source->destination mapping and clears it. """Returns the copy-on-write source->destination mapping and clears it.
Returns: Returns:
Dict[BlockId, List[BlockId]]: A dictionary mapping source List[Tuple[BlockId, BlockId]]: A list mapping source
block indices to lists of destination block indices. block indices to destination block indices.
""" """
return self._cow_tracker.clear_cows() return self._cow_tracker.clear_cows()

View File

@ -1,7 +1,7 @@
"""Token blocks.""" """Token blocks."""
from itertools import takewhile from itertools import takewhile
from os.path import commonprefix from os.path import commonprefix
from typing import Dict, FrozenSet, Iterable, List, Optional from typing import Dict, FrozenSet, Iterable, List, Optional, Tuple
from vllm.core.block.common import (CopyOnWriteTracker, from vllm.core.block.common import (CopyOnWriteTracker,
get_all_blocks_recursively) get_all_blocks_recursively)
@ -337,12 +337,12 @@ class PrefixCachingBlockAllocator(BlockAllocator):
""" """
return self._cow_tracker.cow_block_if_not_appendable(block) return self._cow_tracker.cow_block_if_not_appendable(block)
def clear_copy_on_writes(self) -> Dict[BlockId, List[BlockId]]: def clear_copy_on_writes(self) -> List[Tuple[BlockId, BlockId]]:
"""Returns the copy-on-write source->destination mapping and clears it. """Returns the copy-on-write source->destination mapping and clears it.
Returns: Returns:
Dict[BlockId, List[BlockId]]: A dictionary mapping source List[Tuple[BlockId, BlockId]]: A list mapping source
block indices to lists of destination block indices. block indices to destination block indices.
""" """
return self._cow_tracker.clear_cows() return self._cow_tracker.clear_cows()

View File

@ -5,7 +5,7 @@ from itertools import count, takewhile
from os.path import commonprefix from os.path import commonprefix
from typing import Dict, List, Optional from typing import Dict, List, Optional
from typing import Sequence as GenericSequence from typing import Sequence as GenericSequence
from typing import Set from typing import Set, Tuple
from vllm.block import BlockTable, PhysicalTokenBlock from vllm.block import BlockTable, PhysicalTokenBlock
from vllm.core.evictor_v1 import EvictionPolicy, Evictor, make_evictor from vllm.core.evictor_v1 import EvictionPolicy, Evictor, make_evictor
@ -386,7 +386,7 @@ class BlockSpaceManagerV1(BlockSpaceManager):
self, self,
seq: Sequence, seq: Sequence,
num_lookahead_slots: int = 0, num_lookahead_slots: int = 0,
) -> Dict[int, List[int]]: ) -> List[Tuple[int, int]]:
"""Allocate a physical slot for a new token.""" """Allocate a physical slot for a new token."""
logical_blocks = seq.logical_token_blocks logical_blocks = seq.logical_token_blocks
block_table = self.block_tables[seq.seq_id] block_table = self.block_tables[seq.seq_id]
@ -405,7 +405,7 @@ class BlockSpaceManagerV1(BlockSpaceManager):
# Allocate a new physical block. # Allocate a new physical block.
new_block = self._allocate_last_physical_block(seq) new_block = self._allocate_last_physical_block(seq)
block_table.append(new_block) block_table.append(new_block)
return {} return []
# We want to append the token to the last physical block. # We want to append the token to the last physical block.
last_block = block_table[-1] last_block = block_table[-1]
@ -418,7 +418,7 @@ class BlockSpaceManagerV1(BlockSpaceManager):
maybe_new_block = self._maybe_promote_last_block( maybe_new_block = self._maybe_promote_last_block(
seq, last_block) seq, last_block)
block_table[-1] = maybe_new_block block_table[-1] = maybe_new_block
return {} return []
else: else:
# The last block is shared with other sequences. # The last block is shared with other sequences.
# Copy on Write: Allocate a new block and copy the tokens. # Copy on Write: Allocate a new block and copy the tokens.
@ -426,7 +426,7 @@ class BlockSpaceManagerV1(BlockSpaceManager):
block_table[-1] = new_block block_table[-1] = new_block
self.gpu_allocator.free(last_block) self.gpu_allocator.free(last_block)
return {last_block.block_number: [new_block.block_number]} return [(last_block.block_number, new_block.block_number)]
def fork(self, parent_seq: Sequence, child_seq: Sequence) -> None: def fork(self, parent_seq: Sequence, child_seq: Sequence) -> None:
# NOTE: fork does not allocate a new physical block. # NOTE: fork does not allocate a new physical block.

View File

@ -1,6 +1,7 @@
"""A block manager that manages token blocks.""" """A block manager that manages token blocks."""
from typing import Dict, List, Optional from typing import Dict, List, Optional
from typing import Sequence as GenericSequence from typing import Sequence as GenericSequence
from typing import Tuple
from vllm.core.block.block_table import BlockTable from vllm.core.block.block_table import BlockTable
from vllm.core.block.cpu_gpu_block_allocator import CpuGpuBlockAllocator from vllm.core.block.cpu_gpu_block_allocator import CpuGpuBlockAllocator
@ -166,7 +167,7 @@ class BlockSpaceManagerV2(BlockSpaceManager):
self, self,
seq: Sequence, seq: Sequence,
num_lookahead_slots: int, num_lookahead_slots: int,
) -> Dict[int, List[int]]: ) -> List[Tuple[int, int]]:
block_table = self.block_tables[seq.seq_id] block_table = self.block_tables[seq.seq_id]

View File

@ -2,6 +2,7 @@ import enum
from abc import ABC, abstractmethod from abc import ABC, abstractmethod
from typing import Dict, List from typing import Dict, List
from typing import Sequence as GenericSequence from typing import Sequence as GenericSequence
from typing import Tuple
from vllm.sequence import Sequence, SequenceGroup from vllm.sequence import Sequence, SequenceGroup
@ -54,7 +55,7 @@ class BlockSpaceManager(ABC):
self, self,
seq: Sequence, seq: Sequence,
num_lookahead_slots: int, num_lookahead_slots: int,
) -> Dict[int, List[int]]: ) -> List[Tuple[int, int]]:
pass pass
@abstractmethod @abstractmethod

View File

@ -1027,10 +1027,7 @@ class Scheduler:
for seq in seq_group.get_seqs(status=SequenceStatus.RUNNING): for seq in seq_group.get_seqs(status=SequenceStatus.RUNNING):
cows = self.block_manager.append_slots(seq, num_lookahead_slots) cows = self.block_manager.append_slots(seq, num_lookahead_slots)
blocks_to_copy.extend(cows)
for src, dests in cows.items():
for dest in dests:
blocks_to_copy.append((src, dest))
def _preempt( def _preempt(
self, self,